diff --git a/NOTICE b/NOTICE index 3a1f5496d71cf..7a62b7fed60cd 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Kafka -Copyright 2016 The Apache Software Foundation. +Copyright 2017 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/bin/kafka-broker-api-versions.sh b/bin/kafka-broker-api-versions.sh new file mode 100755 index 0000000000000..4f560a0a60cd5 --- /dev/null +++ b/bin/kafka-broker-api-versions.sh @@ -0,0 +1,17 @@ +#!/bin/bash +# 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. + +exec $(dirname $0)/kafka-run-class.sh kafka.admin.BrokerApiVersionsCommand "$@" diff --git a/bin/windows/kafka-configs.bat b/bin/windows/kafka-configs.bat new file mode 100644 index 0000000000000..8bbbbfa5cfcdd --- /dev/null +++ b/bin/windows/kafka-configs.bat @@ -0,0 +1,17 @@ +@echo off +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. + +%~dp0kafka-run-class.bat kafka.admin.ConfigCommand %* diff --git a/bin/windows/kafka-run-class.bat b/bin/windows/kafka-run-class.bat index 4f85301286589..23263b20ac348 100755 --- a/bin/windows/kafka-run-class.bat +++ b/bin/windows/kafka-run-class.bat @@ -90,7 +90,9 @@ for %%p in (api runtime file json tools) do ( ) rem Classpath addition for release -call :concat %BASE_DIR%\libs\* +for %%i in (%BASE_DIR%\libs\*) do ( + call :concat %%i +) rem Classpath addition for core for %%i in (%BASE_DIR%\core\build\libs\kafka_%SCALA_BINARY_VERSION%*.jar) do ( @@ -174,7 +176,7 @@ IF ["%CLASSPATH%"] EQU [""] ( EXIT /B 2 ) -set COMMAND=%JAVA% %KAFKA_HEAP_OPTS% %KAFKA_JVM_PERFORMANCE_OPTS% %KAFKA_JMX_OPTS% %KAFKA_LOG4J_OPTS% -cp %CLASSPATH% %KAFKA_OPTS% %* +set COMMAND=%JAVA% %KAFKA_HEAP_OPTS% %KAFKA_JVM_PERFORMANCE_OPTS% %KAFKA_JMX_OPTS% %KAFKA_LOG4J_OPTS% -cp "%CLASSPATH%" %KAFKA_OPTS% %* rem echo. rem echo %COMMAND% rem echo. @@ -184,7 +186,7 @@ rem echo. goto :eof :concat IF ["%CLASSPATH%"] EQU [""] ( - set CLASSPATH="%1" + set "CLASSPATH=%1" ) ELSE ( - set CLASSPATH=%CLASSPATH%;"%1" + set "CLASSPATH=%CLASSPATH%;%1" ) diff --git a/build.gradle b/build.gradle index 6f4d2501f7345..32ce14a8249ed 100644 --- a/build.gradle +++ b/build.gradle @@ -177,21 +177,21 @@ subprojects { } jar { - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" } task srcJar(type: Jar) { classifier = 'sources' - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from sourceSets.main.allSource } task javadocJar(type: Jar, dependsOn: javadoc) { classifier 'javadoc' - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from javadoc.destinationDir } @@ -213,15 +213,15 @@ subprojects { if(!sourceSets.test.allSource.isEmpty()) { task testJar(type: Jar) { classifier = 'test' - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from sourceSets.test.output } task testSrcJar(type: Jar, dependsOn: testJar) { classifier = 'test-sources' - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from sourceSets.test.allSource } @@ -234,8 +234,8 @@ subprojects { plugins.withType(ScalaPlugin) { task scaladocJar(type:Jar) { classifier = 'scaladoc' - from '../LICENSE' - from '../NOTICE' + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from scaladoc.destinationDir } @@ -252,7 +252,6 @@ subprojects { "-deprecation", "-unchecked", "-encoding", "utf8", - "-target:jvm-${sourceCompatibility}".toString(), "-Xlog-reflective-calls", "-feature", "-language:postfixOps", @@ -508,11 +507,11 @@ project(':core') { task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs', 'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', - 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', + 'genTopicConfigDocs', ':connect:runtime:genConnectConfigDocs', ':connect:runtime:genConnectTransformationDocs', ':streams:genStreamsConfigDocs'], type: Tar) { classifier = 'site-docs' compression = Compression.GZIP - from project.file("../docs") + from project.file("$rootDir/docs") into 'site-docs' duplicatesStrategy 'exclude' } @@ -520,10 +519,10 @@ project(':core') { tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { into "kafka_${versions.baseScala}-${version}" compression = Compression.GZIP - from(project.file("../bin")) { into "bin/" } - from(project.file("../config")) { into "config/" } - from '../LICENSE' - from '../NOTICE' + from(project.file("$rootDir/bin")) { into "bin/" } + from(project.file("$rootDir/config")) { into "config/" } + from "$rootDir/LICENSE" + from "$rootDir/NOTICE" from(configurations.runtime) { into("libs/") } from(configurations.archives.artifacts.files) { into("libs/") } from(project.siteDocsTar) { into("site-docs/") } @@ -607,12 +606,12 @@ project(':clients') { task determineCommitId { ext.commitId = "unknown" def takeFromHash = 16 - if (file("../.git/HEAD").exists()) { - def headRef = file("../.git/HEAD").text + if (file("$rootDir/.git/HEAD").exists()) { + def headRef = file("$rootDir/.git/HEAD").text if (headRef.contains('ref: ')) { headRef = headRef.replaceAll('ref: ', '').trim() - if (file("../.git/$headRef").exists()) { - commitId = file("../.git/$headRef").text.trim().take(takeFromHash) + if (file("$rootDir/.git/$headRef").exists()) { + commitId = file("$rootDir/.git/$headRef").text.trim().take(takeFromHash) } } else { commitId = headRef.trim().take(takeFromHash) @@ -948,6 +947,13 @@ project(':connect:runtime') { if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "connect_config.html").newOutputStream() } + + task genConnectTransformationDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.connect.tools.TransformationDoc' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "connect_transforms.html").newOutputStream() + } } project(':connect:file') { diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index b68cf988821ab..04f364c6f8c18 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -156,6 +156,7 @@ + diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 75d48abf247ac..428684d4b45f3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -195,8 +195,13 @@ public synchronized boolean containsTopic(String topic) { /** * Updates the cluster metadata. If topic expiry is enabled, expiry time * is set for topics if required and expired topics are removed from the metadata. + * + * @param cluster the cluster containing metadata for topics with valid metadata + * @param unavailableTopics topics which are non-existent or have one or more partitions whose + * leader is not known + * @param now current time in milliseconds */ - public synchronized void update(Cluster cluster, long now) { + public synchronized void update(Cluster cluster, Set unavailableTopics, long now) { Objects.requireNonNull(cluster, "cluster should not be null"); this.needUpdate = false; @@ -219,7 +224,7 @@ else if (expireMs <= now) { } for (Listener listener: listeners) - listener.onMetadataUpdate(cluster); + listener.onMetadataUpdate(cluster, unavailableTopics); String previousClusterId = cluster.clusterResource().clusterId(); @@ -302,7 +307,14 @@ public synchronized void removeListener(Listener listener) { * MetadataUpdate Listener */ public interface Listener { - void onMetadataUpdate(Cluster cluster); + /** + * Callback invoked on metadata update. + * + * @param cluster the cluster containing metadata for topics with valid metadata + * @param unavailableTopics topics which are non-existent or have one or more partitions whose + * leader is not known + */ + void onMetadataUpdate(Cluster cluster, Set unavailableTopics); } private synchronized void requestUpdateForNewTopics() { 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 5e97eacd970a6..1e8dbd37c99c5 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -14,7 +14,6 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; -import org.apache.kafka.common.errors.ObsoleteBrokerException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.Selectable; @@ -289,20 +288,17 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long // the case when sending the initial ApiVersionRequest which fetches the version // information itself. It is also the case when discoverBrokerVersions is set to false. if (versionInfo == null) { - if ((!discoverBrokerVersions) && (log.isTraceEnabled())) - log.trace("No version information found when sending message of type {} to node {}", - clientRequest.apiKey(), nodeId); + if (discoverBrokerVersions && log.isTraceEnabled()) + log.trace("No version information found when sending message of type {} to node {}. " + + "Assuming version {}.", clientRequest.apiKey(), nodeId, builder.version()); } else { short version = versionInfo.usableVersion(clientRequest.apiKey()); - if (log.isTraceEnabled()) - log.trace("When sending message of type {} to node {}, the best usable version is {}", - clientRequest.apiKey(), nodeId, version); builder.setVersion(version); } // The call to build may also throw UnsupportedVersionException, if there are essential // fields that cannot be represented in the chosen version. request = builder.build(); - } catch (ObsoleteBrokerException | UnsupportedVersionException e) { + } catch (UnsupportedVersionException e) { // If the version is not supported, skip sending the request over the wire. // Instead, simply add it to the local queue of aborted requests. log.debug("Version mismatch when attempting to send {} to {}", @@ -314,8 +310,15 @@ private void doSend(ClientRequest clientRequest, boolean isInternalRequest, long return; } RequestHeader header = clientRequest.makeHeader(); - if (log.isTraceEnabled()) - log.trace("Sending {} to node {}", request, nodeId); + if (log.isDebugEnabled()) { + int latestClientVersion = ProtoUtils.latestVersion(clientRequest.apiKey().id); + if (header.apiVersion() == latestClientVersion) { + log.trace("Sending {} to node {}.", request, nodeId); + } else { + log.debug("Using older server API v{} to send {} to node {}.", + header.apiVersion(), request, nodeId); + } + } Send send = request.toSend(nodeId, header); InFlightRequest inFlightRequest = new InFlightRequest( header, @@ -424,14 +427,23 @@ public Node leastLoadedNode(long now) { int currInflight = this.inFlightRequests.inFlightRequestCount(node.idString()); if (currInflight == 0 && this.connectionStates.isReady(node.idString())) { // if we find an established connection with no in-flight requests we can stop right away + log.trace("Found least loaded node {} connected with no in-flight requests", node); return node; } else if (!this.connectionStates.isBlackedOut(node.idString(), now) && currInflight < inflight) { // otherwise if this is the best we have found so far, record that inflight = currInflight; found = node; + } else if (log.isTraceEnabled()) { + log.trace("Removing node {} from least loaded node selection: is-blacked-out: {}, in-flight-requests: {}", + node, this.connectionStates.isBlackedOut(node.idString(), now), currInflight); } } + if (found != null) + log.trace("Found least loaded node {}", found); + else + log.trace("Least loaded node selection failed to find an available node"); + return found; } @@ -697,7 +709,7 @@ public void handleCompletedMetadataResponse(RequestHeader requestHeader, long no // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being // created which means we will get errors and no nodes until it exists if (cluster.nodes().size() > 0) { - this.metadata.update(cluster, now); + this.metadata.update(cluster, response.unavailableTopics(), now); } else { log.trace("Ignoring empty metadata response with correlation id {}.", requestHeader.correlationId()); this.metadata.failedUpdate(now); diff --git a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java index 6acbb6318374c..906c2264c55ce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java +++ b/clients/src/main/java/org/apache/kafka/clients/NodeApiVersions.java @@ -19,65 +19,139 @@ import org.apache.kafka.common.utils.Utils; import java.util.Collection; +import java.util.Collections; import java.util.EnumMap; +import java.util.LinkedList; +import java.util.List; import java.util.Map; +import java.util.NoSuchElementException; import java.util.TreeMap; +/** + * An internal class which represents the API versions supported by a particular node. + */ public class NodeApiVersions { - private final Collection apiVersions; + private static final short NODE_TOO_OLD = (short) -1; + private static final short NODE_TOO_NEW = (short) -2; + private final Collection nodeApiVersions; - // An array of the usable versions of each API, indexed by ApiKeys ID. + /** + * An array of the usable versions of each API, indexed by the ApiKeys ID. + */ private final Map usableVersions = new EnumMap<>(ApiKeys.class); - public NodeApiVersions(Collection apiVersions) { - this.apiVersions = apiVersions; - for (ApiVersion apiVersion : apiVersions) { - int apiKeyId = apiVersion.apiKey; + /** + * Create a NodeApiVersions object with the current ApiVersions. + * + * @return A new NodeApiVersions object. + */ + public static NodeApiVersions create() { + return create(Collections.emptyList()); + } + + /** + * Create a NodeApiVersions object. + * + * @param overrides API versions to override. Any ApiVersion not specified here will be set to the current client + * value. + * @return A new NodeApiVersions object. + */ + public static NodeApiVersions create(Collection overrides) { + List apiVersions = new LinkedList<>(overrides); + for (ApiKeys apiKey : ApiKeys.values()) { + boolean exists = false; + for (ApiVersion apiVersion : apiVersions) { + if (apiVersion.apiKey == apiKey.id) { + exists = true; + break; + } + } + if (!exists) { + apiVersions.add(new ApiVersion(apiKey.id, ProtoUtils.oldestVersion(apiKey.id), + ProtoUtils.latestVersion(apiKey.id))); + } + } + return new NodeApiVersions(apiVersions); + } + + public NodeApiVersions(Collection nodeApiVersions) { + this.nodeApiVersions = nodeApiVersions; + for (ApiVersion nodeApiVersion : nodeApiVersions) { + int nodeApiKey = nodeApiVersion.apiKey; // Newer brokers may support ApiKeys we don't know about, ignore them - if (ApiKeys.hasId(apiKeyId)) { - short version = Utils.min(ProtoUtils.latestVersion(apiKeyId), apiVersion.maxVersion); - if (version >= apiVersion.minVersion && version >= ProtoUtils.oldestVersion(apiKeyId)) - usableVersions.put(ApiKeys.forId(apiKeyId), version); + if (ApiKeys.hasId(nodeApiKey)) { + short v = Utils.min(ProtoUtils.latestVersion(nodeApiKey), nodeApiVersion.maxVersion); + if (v < nodeApiVersion.minVersion) { + usableVersions.put(ApiKeys.forId(nodeApiKey), NODE_TOO_NEW); + } else if (v < ProtoUtils.oldestVersion(nodeApiKey)) { + usableVersions.put(ApiKeys.forId(nodeApiKey), NODE_TOO_OLD); + } else { + usableVersions.put(ApiKeys.forId(nodeApiKey), v); + } } } } /** - * Return the most recent version supported by both the client and the server. + * Return the most recent version supported by both the node and the local software. */ public short usableVersion(ApiKeys apiKey) { Short usableVersion = usableVersions.get(apiKey); - if (usableVersion == null) { - throw new UnsupportedVersionException("The client cannot send an " + - "API request of type " + apiKey + ", because the " + - "server does not understand any of the versions this client supports."); - } - return usableVersion; + if (usableVersion == null) + throw new UnsupportedVersionException("The broker does not support " + apiKey); + else if (usableVersion == NODE_TOO_OLD) + throw new UnsupportedVersionException("The broker is too old to support " + apiKey + + " version " + ProtoUtils.oldestVersion(apiKey.id)); + else if (usableVersion == NODE_TOO_NEW) + throw new UnsupportedVersionException("The broker is too new to support " + apiKey + + " version " + ProtoUtils.latestVersion(apiKey.id)); + else + return usableVersion; } /** + * Convert the object to a string with no linebreaks.

+ * * This toString method is relatively expensive, so avoid calling it unless debug logging is turned on. */ @Override public String toString() { + return toString(false); + } + + /** + * Convert the object to a string. + * + * @param lineBreaks True if we should add a linebreak after each api. + */ + public String toString(boolean lineBreaks) { // The apiVersion collection may not be in sorted order. We put it into // a TreeMap before printing it out to ensure that we always print in // ascending order. TreeMap apiKeysText = new TreeMap<>(); - for (ApiVersion apiVersion : this.apiVersions) + for (ApiVersion apiVersion : this.nodeApiVersions) apiKeysText.put(apiVersion.apiKey, apiVersionToText(apiVersion)); - // Also handle the case where some apiKey types are - // unknown, which may happen when either the client or server is newer. + // Also handle the case where some apiKey types are not specified at all in the given ApiVersions, + // which may happen when the remote is too old. for (ApiKeys apiKey : ApiKeys.values()) { if (!apiKeysText.containsKey(apiKey.id)) { StringBuilder bld = new StringBuilder(); bld.append(apiKey.name).append("("). - append(apiKey.id).append("): ").append("UNSUPPORTED"); + append(apiKey.id).append("): ").append("UNSUPPORTED"); apiKeysText.put(apiKey.id, bld.toString()); } } - return "{" + Utils.join(apiKeysText.values(), ", ") + "}"; + String separator = lineBreaks ? ",\n\t" : ", "; + StringBuilder bld = new StringBuilder(); + bld.append("("); + if (lineBreaks) + bld.append("\n\t"); + bld.append(Utils.join(apiKeysText.values(), separator)); + if (lineBreaks) + bld.append("\n"); + bld.append(")"); + return bld.toString(); } private String apiVersionToText(ApiVersion apiVersion) { @@ -85,25 +159,35 @@ private String apiVersionToText(ApiVersion apiVersion) { ApiKeys apiKey = null; if (ApiKeys.hasId(apiVersion.apiKey)) { apiKey = ApiKeys.forId(apiVersion.apiKey); - } - if (apiKey != null) { bld.append(apiKey.name).append("(").append(apiKey.id).append("): "); } else { - bld.append("UNKNOWN(").append(apiKey.id).append("): "); + bld.append("UNKNOWN(").append(apiVersion.apiKey).append("): "); } + if (apiVersion.minVersion == apiVersion.maxVersion) { bld.append(apiVersion.minVersion); } else { bld.append(apiVersion.minVersion).append(" to ").append(apiVersion.maxVersion); } + if (apiKey != null) { Short usableVersion = usableVersions.get(apiKey); - if (usableVersion == null) { - bld.append(" [usable: NONE]"); - } else { + if (usableVersion == NODE_TOO_OLD) + bld.append(" [unusable: node too old]"); + else if (usableVersion == NODE_TOO_NEW) + bld.append(" [unusable: node too new]"); + else bld.append(" [usable: ").append(usableVersion).append("]"); - } } return bld.toString(); } + + public ApiVersion apiVersion(ApiKeys apiKey) { + for (ApiVersion nodeApiVersion : nodeApiVersions) { + if (nodeApiVersion.apiKey == apiKey.id) { + return nodeApiVersion; + } + } + throw new NoSuchElementException(); + } } 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 ed809a9f401ad..1b335173b8884 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 @@ -426,6 +426,10 @@ public static Properties addDeserializerToConfig(Properties properties, super(CONFIG, props); } + ConsumerConfig(Map props, boolean doLog) { + super(CONFIG, props, doLog); + } + public static Set configNames() { return CONFIG.names(); } 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 938d22bbbece5..a4265abb76eda 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 @@ -86,7 +86,7 @@ public interface ConsumerRebalanceListener { * * @param partitions The list of partitions that were assigned to the consumer on the last rebalance */ - public void onPartitionsRevoked(Collection partitions); + void onPartitionsRevoked(Collection partitions); /** * A callback method the user can implement to provide handling of customized offsets on completion of a successful @@ -100,5 +100,5 @@ public interface ConsumerRebalanceListener { * @param partitions The list of partitions that are now assigned to the consumer (may include partitions previously * assigned to the consumer) */ - public void onPartitionsAssigned(Collection partitions); + void onPartitionsAssigned(Collection partitions); } 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 1b033e9e06639..64d64ba639f84 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 @@ -75,6 +75,13 @@ * Failure to close the consumer after use will leak these connections. * The consumer is not thread-safe. See Multi-threaded Processing for more details. * + *

Cross-Version Compatibility

+ * This client can communicate with brokers that are version 0.10.0 or newer. Older or newer brokers may not support + * certain features. For example, 0.10.0 brokers do not support offsetsForTimes, because this feature was added + * in version 0.10.1. You will receive an {@link org.apache.kafka.common.errors.UnsupportedVersionException} + * when invoking an API that is not available on the running broker version. + *

+ * *

Offsets and Consumer Position

* Kafka maintains a numerical offset for each record in a partition. This offset acts as a unique identifier of * a record within that partition, and also denotes the position of the consumer in the partition. For example, a consumer @@ -497,7 +504,6 @@ * There are many possible variations on this approach. For example each processor thread can have its own queue, and * the consumer threads can hash into these queues using the TopicPartition to ensure in-order consumption and simplify * commit. - * */ public class KafkaConsumer implements Consumer { @@ -624,7 +630,7 @@ private KafkaConsumer(ConsumerConfig config, // load interceptors and make sure they get clientId Map userProvidedConfigs = config.originals(); userProvidedConfigs.put(ConsumerConfig.CLIENT_ID_CONFIG, clientId); - List> interceptorList = (List) (new ConsumerConfig(userProvidedConfigs)).getConfiguredInstances(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, + List> interceptorList = (List) (new ConsumerConfig(userProvidedConfigs, false)).getConfiguredInstances(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, ConsumerInterceptor.class); this.interceptors = interceptorList.isEmpty() ? null : new ConsumerInterceptors<>(interceptorList); if (keyDeserializer == null) { @@ -646,7 +652,7 @@ private KafkaConsumer(ConsumerConfig config, ClusterResourceListeners clusterResourceListeners = configureClusterResourceListeners(keyDeserializer, valueDeserializer, reporters, interceptorList); this.metadata = new Metadata(retryBackoffMs, config.getLong(ConsumerConfig.METADATA_MAX_AGE_CONFIG), false, clusterResourceListeners); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), 0); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), 0); String metricGrpPrefix = "consumer"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values()); NetworkClient netClient = new NetworkClient( @@ -663,7 +669,7 @@ private KafkaConsumer(ConsumerConfig config, this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG)); OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(Locale.ROOT)); - this.subscriptions = new SubscriptionState(offsetResetStrategy, metrics); + this.subscriptions = new SubscriptionState(offsetResetStrategy); List assignors = config.getConfiguredInstances( ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, PartitionAssignor.class); @@ -679,7 +685,6 @@ private KafkaConsumer(ConsumerConfig config, metricGrpPrefix, this.time, retryBackoffMs, - new ConsumerCoordinator.DefaultOffsetCommitCallback(), config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG), config.getInt(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG), this.interceptors, @@ -995,9 +1000,8 @@ public ConsumerRecords poll(long timeout) { // // NOTE: since the consumed position has already been updated, we must not allow // wakeups or any other errors to be triggered prior to returning the fetched records. - if (fetcher.sendFetches() > 0) { + if (fetcher.sendFetches() > 0 || client.pendingRequestCount() > 0) client.pollNoWakeup(); - } if (this.interceptors == null) return new ConsumerRecords<>(records); @@ -1083,7 +1087,7 @@ public boolean shouldBlock() { public void commitSync() { acquire(); try { - commitSync(subscriptions.allConsumed()); + coordinator.commitOffsetsSync(subscriptions.allConsumed(), Long.MAX_VALUE); } finally { release(); } @@ -1117,7 +1121,7 @@ public void commitSync() { public void commitSync(final Map offsets) { acquire(); try { - coordinator.commitOffsetsSync(offsets, Long.MAX_VALUE); + coordinator.commitOffsetsSync(new HashMap<>(offsets), Long.MAX_VALUE); } finally { release(); } @@ -1437,6 +1441,8 @@ public Set paused() { * than or equal to the target timestamp. {@code null} will be returned for the partition if there is no * such message. * @throws IllegalArgumentException if the target timestamp is negative. + * @throws org.apache.kafka.common.errors.UnsupportedVersionException if the broker does not support looking up + * the offsets by timestamp. */ @Override public Map offsetsForTimes(Map timestampsToSearch) { @@ -1488,7 +1494,7 @@ public Map endOffsets(Collection partition * If auto-commit is enabled, this will commit the current offsets if possible within the default * timeout. See {@link #close(long, TimeUnit)} for details. Note that {@link #wakeup()} * cannot be used to interrupt close. - * + * * @throws org.apache.kafka.common.errors.InterruptException if the calling thread is interrupted * before or while this function is called */ @@ -1584,9 +1590,9 @@ private void updateFetchPositions(Set partitions) { // the user is manually assigning partitions and managing their own offsets). fetcher.resetOffsetsIfNeeded(partitions); - if (!subscriptions.hasAllFetchPositions()) { - // if we still don't have offsets for all partitions, then we should either seek - // to the last committed position or reset using the auto reset policy + if (!subscriptions.hasAllFetchPositions(partitions)) { + // if we still don't have offsets for the given partitions, then we should either + // seek to the last committed position or reset using the auto reset policy // first refresh commits for all assigned partitions coordinator.refreshCommittedOffsetsIfNeeded(); 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 95e3830952db0..a88f4324ccb2d 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 @@ -34,7 +34,6 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import java.util.regex.Pattern; -import org.apache.kafka.common.metrics.Metrics; /** @@ -59,7 +58,7 @@ public class MockConsumer implements Consumer { private AtomicBoolean wakeup; public MockConsumer(OffsetResetStrategy offsetResetStrategy) { - this.subscriptions = new SubscriptionState(offsetResetStrategy, new Metrics()); + this.subscriptions = new SubscriptionState(offsetResetStrategy); this.partitions = new HashMap<>(); this.records = new HashMap<>(); this.paused = new HashSet<>(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java index f74a333733bf0..2f95291112ac2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndTimestamp.java @@ -20,11 +20,6 @@ /** * A container class for offset and timestamp. - * - * Offset must be non-negative. - * - * The timestamp should never be negative, unless it is invalid. This could happen when - * handling a response from a broker that doesn't support KIP-79. */ public final class OffsetAndTimestamp { private final long timestamp; @@ -32,7 +27,9 @@ public final class OffsetAndTimestamp { public OffsetAndTimestamp(long offset, long timestamp) { this.offset = offset; + assert this.offset >= 0; this.timestamp = timestamp; + assert this.timestamp >= 0; } public long timestamp() { @@ -45,7 +42,7 @@ public long offset() { @Override public String toString() { - return "{timestamp=" + timestamp + ", offset=" + offset + "}"; + return "(timestamp=" + timestamp + ", offset=" + offset + ")"; } @Override 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 7d77c0bcbd38a..b72769ec0cc12 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 @@ -43,6 +43,7 @@ import org.apache.kafka.common.requests.OffsetCommitRequest; import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; +import org.apache.kafka.common.utils.KafkaThread; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -211,6 +212,7 @@ protected synchronized boolean ensureCoordinatorReady(long startTimeMs, long tim if (remainingMs <= 0) break; + log.debug("Coordinator discovery failed for group {}, refreshing metadata", groupId); client.awaitMetadataUpdate(remainingMs); } else throw future.exception(); @@ -236,6 +238,7 @@ protected synchronized RequestFuture lookupCoordinator() { if (node == null) { // TODO: If there are no brokers left, perhaps we should use the bootstrap set // from configuration? + log.debug("No broker available to send GroupCoordinator request for group {}", groupId); return RequestFuture.noBrokersAvailable(); } else findCoordinatorFuture = sendGroupCoordinatorRequest(node); @@ -312,6 +315,19 @@ private synchronized void disableHeartbeatThread() { heartbeatThread.disable(); } + private void closeHeartbeatThread() { + if (heartbeatThread != null) { + heartbeatThread.close(); + + try { + heartbeatThread.join(); + } catch (InterruptedException e) { + log.warn("Interrupted while waiting for consumer heartbeat thread to close"); + throw new InterruptException(e); + } + } + } + // visible for testing. Joins the group without starting the heartbeat thread. void joinGroupIfNeeded() { while (needRejoin() || rejoinIncomplete()) { @@ -419,7 +435,7 @@ private class JoinGroupResponseHandler extends CoordinatorResponseHandler future) { Errors error = Errors.forCode(joinResponse.errorCode()); if (error == Errors.NONE) { - log.debug("Received successful join group response for group {}: {}", groupId, joinResponse); + log.debug("Received successful JoinGroup response for group {}: {}", groupId, joinResponse); sensors.joinLatency.record(response.requestLatencyMs()); synchronized (AbstractCoordinator.this) { @@ -542,7 +558,7 @@ public void handle(SyncGroupResponse syncResponse, */ private RequestFuture sendGroupCoordinatorRequest(Node node) { // initiate the group metadata request - log.debug("Sending coordinator request for group {} to broker {}", groupId, node); + log.debug("Sending GroupCoordinator request for group {} to broker {}", groupId, node); GroupCoordinatorRequest.Builder requestBuilder = new GroupCoordinatorRequest.Builder(this.groupId); return client.send(node, requestBuilder) @@ -553,7 +569,7 @@ private class GroupCoordinatorResponseHandler extends RequestFutureAdapter future) { - log.debug("Received group coordinator response {}", resp); + log.debug("Received GroupCoordinator response {} for group {}", resp, groupId); GroupCoordinatorResponse groupCoordinatorResponse = (GroupCoordinatorResponse) resp.responseBody(); // use MAX_VALUE - node.id as the coordinator id to mimic separate connections @@ -649,19 +665,26 @@ public synchronized void close() { close(0); } - protected synchronized void close(long timeoutMs) { - if (heartbeatThread != null) - heartbeatThread.close(); - maybeLeaveGroup(); - - // At this point, there may be pending commits (async commits or sync commits that were - // interrupted using wakeup) and the leave group request which have been queued, but not - // yet sent to the broker. Wait up to close timeout for these pending requests to be processed. - // If coordinator is not known, requests are aborted. - Node coordinator = coordinator(); - if (coordinator != null && !client.awaitPendingRequests(coordinator, timeoutMs)) - log.warn("Close timed out with {} pending requests to coordinator, terminating client connections for group {}.", - client.pendingRequestCount(coordinator), groupId); + protected void close(long timeoutMs) { + try { + closeHeartbeatThread(); + } finally { + + // Synchronize after closing the heartbeat thread since heartbeat thread + // needs this lock to complete and terminate after close flag is set. + synchronized (this) { + maybeLeaveGroup(); + + // At this point, there may be pending commits (async commits or sync commits that were + // interrupted using wakeup) and the leave group request which have been queued, but not + // yet sent to the broker. Wait up to close timeout for these pending requests to be processed. + // If coordinator is not known, requests are aborted. + Node coordinator = coordinator(); + if (coordinator != null && !client.awaitPendingRequests(coordinator, timeoutMs)) + log.warn("Close timed out with {} pending requests to coordinator, terminating client connections for group {}.", + client.pendingRequestCount(coordinator), groupId); + } + } } /** @@ -671,6 +694,7 @@ public synchronized void maybeLeaveGroup() { if (!coordinatorUnknown() && state != MemberState.UNJOINED && generation != Generation.NO_GENERATION) { // this is a minimal effort attempt to leave the group. we do not // attempt any resending if the request fails or times out. + log.debug("Sending LeaveGroup request to coordinator {} for group {}", coordinator, groupId); LeaveGroupRequest.Builder request = new LeaveGroupRequest.Builder(groupId, generation.memberId); client.send(coordinator, request) @@ -697,6 +721,7 @@ public void handle(LeaveGroupResponse leaveResponse, RequestFuture future) // visible for testing synchronized RequestFuture sendHeartbeatRequest() { + log.debug("Sending Heartbeat request for group {} to coordinator {}", groupId, coordinator); HeartbeatRequest.Builder requestBuilder = new HeartbeatRequest.Builder(this.groupId, this.generation.generationId, this.generation.memberId); return client.send(coordinator, requestBuilder) @@ -709,24 +734,24 @@ public void handle(HeartbeatResponse heartbeatResponse, RequestFuture futu sensors.heartbeatLatency.record(response.requestLatencyMs()); Errors error = Errors.forCode(heartbeatResponse.errorCode()); if (error == Errors.NONE) { - log.debug("Received successful heartbeat response for group {}", groupId); + log.debug("Received successful Heartbeat response for group {}", groupId); future.complete(null); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR_FOR_GROUP) { - log.debug("Attempt to heart beat failed for group {} since coordinator {} is either not started or not valid.", + log.debug("Attempt to heartbeat failed for group {} since coordinator {} is either not started or not valid.", groupId, coordinator()); coordinatorDead(); future.raise(error); } else if (error == Errors.REBALANCE_IN_PROGRESS) { - log.debug("Attempt to heart beat failed for group {} since it is rebalancing.", groupId); + log.debug("Attempt to heartbeat failed for group {} since it is rebalancing.", groupId); requestRejoin(); future.raise(Errors.REBALANCE_IN_PROGRESS); } else if (error == Errors.ILLEGAL_GENERATION) { - log.debug("Attempt to heart beat failed for group {} since generation id is not legal.", groupId); + log.debug("Attempt to heartbeat failed for group {} since generation id is not legal.", groupId); resetGeneration(); future.raise(Errors.ILLEGAL_GENERATION); } else if (error == Errors.UNKNOWN_MEMBER_ID) { - log.debug("Attempt to heart beat failed for group {} since member id is not valid.", groupId); + log.debug("Attempt to heartbeat failed for group {} since member id is not valid.", groupId); resetGeneration(); future.raise(Errors.UNKNOWN_MEMBER_ID); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { @@ -818,18 +843,18 @@ public double measure(MetricConfig config, long now) { } } - private class HeartbeatThread extends Thread { + private class HeartbeatThread extends KafkaThread { private boolean enabled = false; private boolean closed = false; private AtomicReference failed = new AtomicReference<>(null); - HeartbeatThread() { - super("kafka-coordinator-heartbeat-thread" + (groupId.isEmpty() ? "" : " | " + groupId)); - setDaemon(true); + private HeartbeatThread() { + super("kafka-coordinator-heartbeat-thread" + (groupId.isEmpty() ? "" : " | " + groupId), true); } public void enable() { synchronized (AbstractCoordinator.this) { + log.trace("Enabling heartbeat thread for group {}", groupId); this.enabled = true; heartbeat.resetTimeouts(time.milliseconds()); AbstractCoordinator.this.notify(); @@ -838,6 +863,7 @@ public void enable() { public void disable() { synchronized (AbstractCoordinator.this) { + log.trace("Disabling heartbeat thread for group {}", groupId); this.enabled = false; } } @@ -860,6 +886,7 @@ private RuntimeException failureCause() { @Override public void run() { try { + log.debug("Heartbeat thread for group {} started", groupId); while (true) { synchronized (AbstractCoordinator.this) { if (closed) @@ -936,6 +963,8 @@ public void onFailure(RuntimeException e) { } catch (RuntimeException e) { log.error("Heartbeat thread for group {} failed due to unexpected error" , groupId, e); this.failed.set(e); + } finally { + log.debug("Heartbeat thread for group {} has closed", groupId); } } 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 7c463d10c9e8c..2e37636af4466 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 @@ -49,6 +49,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; @@ -100,7 +101,6 @@ public ConsumerCoordinator(ConsumerNetworkClient client, String metricGrpPrefix, Time time, long retryBackoffMs, - OffsetCommitCallback defaultOffsetCommitCallback, boolean autoCommitEnabled, int autoCommitIntervalMs, ConsumerInterceptors interceptors, @@ -117,7 +117,7 @@ public ConsumerCoordinator(ConsumerNetworkClient client, this.metadata = metadata; this.metadataSnapshot = new MetadataSnapshot(subscriptions, metadata.fetch()); this.subscriptions = subscriptions; - this.defaultOffsetCommitCallback = defaultOffsetCommitCallback; + this.defaultOffsetCommitCallback = new DefaultOffsetCommitCallback(); this.autoCommitEnabled = autoCommitEnabled; this.autoCommitIntervalMs = autoCommitIntervalMs; this.assignors = assignors; @@ -155,7 +155,7 @@ public void updatePatternSubscription(Cluster cluster) { final Set topicsToSubscribe = new HashSet<>(); for (String topic : cluster.topics()) - if (subscriptions.getSubscribedPattern().matcher(topic).matches() && + if (subscriptions.subscribedPattern().matcher(topic).matches() && !(excludeInternalTopics && cluster.internalTopics().contains(topic))) topicsToSubscribe.add(topic); @@ -169,7 +169,7 @@ public void updatePatternSubscription(Cluster cluster) { private void addMetadataListener() { this.metadata.addListener(new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { // if we encounter any unauthorized topics, raise an exception to the user if (!cluster.unauthorizedTopics().isEmpty()) throw new TopicAuthorizationException(new HashSet<>(cluster.unauthorizedTopics())); @@ -183,6 +183,9 @@ public void onMetadataUpdate(Cluster cluster) { if (!snapshot.equals(metadataSnapshot)) metadataSnapshot = snapshot; } + + if (!Collections.disjoint(metadata.topics(), unavailableTopics)) + metadata.requestUpdate(); } }); } @@ -216,6 +219,32 @@ protected void onJoinComplete(int generation, // update partition assignment subscriptions.assignFromSubscribed(assignment.partitions()); + // check if the assignment contains some topics that were not in the original + // subscription, if yes we will obey what leader has decided and add these topics + // into the subscriptions as long as they still match the subscribed pattern + // + // TODO this part of the logic should be removed once we allow regex on leader assign + Set addedTopics = new HashSet<>(); + for (TopicPartition tp : subscriptions.assignedPartitions()) { + if (!joinedSubscription.contains(tp.topic())) + addedTopics.add(tp.topic()); + } + + if (!addedTopics.isEmpty()) { + Set newSubscription = new HashSet<>(subscriptions.subscription()); + Set newJoinedSubscription = new HashSet<>(joinedSubscription); + newSubscription.addAll(addedTopics); + newJoinedSubscription.addAll(addedTopics); + + this.subscriptions.subscribeFromPattern(newSubscription); + this.joinedSubscription = newJoinedSubscription; + } + + // update the metadata and enforce a refresh to make sure the fetcher can start + // fetching data in the next iteration + this.metadata.setTopics(subscriptions.groupSubscription()); + client.ensureFreshMetadata(); + // give the assignor a chance to update internal state based on the received assignment assignor.onAssignment(assignment); @@ -307,13 +336,46 @@ protected Map performAssignment(String leaderId, client.ensureFreshMetadata(); isLeader = true; - assignmentSnapshot = metadataSnapshot; log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", groupId, assignor.name(), subscriptions); Map assignment = assignor.assign(metadata.fetch(), subscriptions); + // user-customized assignor may have created some topics that are not in the subscription list + // and assign their partitions to the members; in this case we would like to update the leader's + // own metadata with the newly added topics so that it will not trigger a subsequent rebalance + // 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 PartitionAssingor API to better support this case. + Set assignedTopics = new HashSet<>(); + for (Assignment assigned : assignment.values()) { + for (TopicPartition tp : assigned.partitions()) + assignedTopics.add(tp.topic()); + } + + if (!assignedTopics.containsAll(allSubscribedTopics)) { + Set notAssignedTopics = new HashSet<>(allSubscribedTopics); + notAssignedTopics.removeAll(assignedTopics); + log.warn("The following subscribed topics are not assigned to any members in the group {} : {} ", groupId, + notAssignedTopics); + } + + if (!allSubscribedTopics.containsAll(assignedTopics)) { + Set newlyAddedTopics = new HashSet<>(assignedTopics); + newlyAddedTopics.removeAll(allSubscribedTopics); + log.info("The following not-subscribed topics are assigned to group {}, and their metadata will be " + + "fetched from the brokers : {}", groupId, newlyAddedTopics); + + allSubscribedTopics.addAll(assignedTopics); + this.subscriptions.groupSubscribe(allSubscribedTopics); + metadata.setTopics(this.subscriptions.groupSubscription()); + client.ensureFreshMetadata(); + } + + assignmentSnapshot = metadataSnapshot; + log.debug("Finished assignment for group {}: {}", groupId, assignment); Map groupAssignment = new HashMap<>(); @@ -430,7 +492,6 @@ void invokeCompletedOffsetCommitCallbacks() { } } - public void commitOffsetsAsync(final Map offsets, final OffsetCommitCallback callback) { invokeCompletedOffsetCommitCallbacks(); @@ -555,15 +616,19 @@ public void maybeAutoCommitOffsetsNow() { } private void doAutoCommitOffsetsAsync() { - commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() { + Map allConsumedOffsets = subscriptions.allConsumed(); + log.debug("Sending asynchronous auto-commit of offsets {} for group {}", allConsumedOffsets, groupId); + + commitOffsetsAsync(allConsumedOffsets, new OffsetCommitCallback() { @Override public void onComplete(Map offsets, Exception exception) { if (exception != null) { - log.warn("Auto offset commit failed for group {}: {}", groupId, exception.getMessage()); + log.warn("Auto-commit of offsets {} failed for group {}: {}", offsets, groupId, + exception.getMessage()); if (exception instanceof RetriableException) nextAutoCommitDeadline = Math.min(time.milliseconds() + retryBackoffMs, nextAutoCommitDeadline); } else { - log.debug("Completed autocommit of offsets {} for group {}", offsets, groupId); + log.debug("Completed auto-commit of offsets {} for group {}", offsets, groupId); } } }); @@ -571,25 +636,30 @@ public void onComplete(Map offsets, Exception private void maybeAutoCommitOffsetsSync(long timeoutMs) { if (autoCommitEnabled) { + Map allConsumedOffsets = subscriptions.allConsumed(); try { - if (!commitOffsetsSync(subscriptions.allConsumed(), timeoutMs)) - log.debug("Automatic commit of offsets {} for group {} timed out before completion", subscriptions.allConsumed(), groupId); + log.debug("Sending synchronous auto-commit of offsets {} for group {}", allConsumedOffsets, groupId); + if (!commitOffsetsSync(allConsumedOffsets, timeoutMs)) + log.debug("Auto-commit of offsets {} for group {} timed out before completion", + allConsumedOffsets, groupId); } catch (WakeupException | InterruptException e) { - log.debug("Automatic commit of offsets {} for group {} was interrupted before completion", subscriptions.allConsumed(), groupId); + log.debug("Auto-commit of offsets {} for group {} was interrupted before completion", + allConsumedOffsets, groupId); // rethrow wakeups since they are triggered by the user throw e; } catch (Exception e) { // consistent with async auto-commit failures, we do not propagate the exception - log.warn("Auto offset commit failed for group {}: {}", groupId, e.getMessage()); + log.warn("Auto-commit of offsets {} failed for group {}: {}", allConsumedOffsets, groupId, + e.getMessage()); } } } - public static class DefaultOffsetCommitCallback implements OffsetCommitCallback { + private class DefaultOffsetCommitCallback implements OffsetCommitCallback { @Override public void onComplete(Map offsets, Exception exception) { if (exception != null) - log.error("Offset commit failed.", exception); + log.error("Offset commit with offsets {} failed for group {}", offsets, groupId, exception); } } @@ -637,7 +707,7 @@ private RequestFuture sendOffsetCommitRequest(final Map offsets; - public OffsetCommitResponseHandler(Map offsets) { + private OffsetCommitResponseHandler(Map offsets) { this.offsets = offsets; } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index ea92ab76d4b15..e5c5cf67daffd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -90,8 +90,7 @@ public ConsumerNetworkClient(KafkaClient client, * @param requestBuilder A builder for the request payload * @return A future which indicates the result of the send. */ - public RequestFuture send(Node node, - AbstractRequest.Builder requestBuilder) { + public RequestFuture send(Node node, AbstractRequest.Builder requestBuilder) { long now = time.milliseconds(); RequestFutureCompletionHandler completionHandler = new RequestFutureCompletionHandler(); ClientRequest clientRequest = client.newClientRequest(node.idString(), requestBuilder, now, true, @@ -157,6 +156,7 @@ public void ensureFreshMetadata() { public void wakeup() { // wakeup should be safe without holding the client lock since it simply delegates to // Selector's wakeup, which is threadsafe + log.trace("Received user wakeup"); this.wakeup.set(true); this.client.wakeup(); } @@ -406,6 +406,7 @@ private boolean trySend(long now) { private void maybeTriggerWakeup() { if (wakeupDisabledCount == 0 && wakeup.get()) { + log.trace("Raising wakeup exception in response to user wakeup"); wakeup.set(false); throw new WakeupException(); } @@ -446,7 +447,7 @@ public void close() throws IOException { /** * Find whether a previous connection has failed. Note that the failure state will persist until either - * {@link #tryConnect(Node)} or {@link #send(Node, ApiKeys, AbstractRequest)} has been called. + * {@link #tryConnect(Node)} or {@link #send(Node, AbstractRequest.Builder)} has been called. * @param node Node to connect to if possible */ public boolean connectionFailed(Node node) { @@ -457,7 +458,7 @@ public boolean connectionFailed(Node node) { /** * Initiate a connection if currently possible. This is only really useful for resetting the failed - * status of a socket. If there is an actual request to send, then {@link #send(Node, ApiKeys, AbstractRequest)} + * status of a socket. If there is an actual request to send, then {@link #send(Node, AbstractRequest.Builder)} * should be used. * @param node The node to connect to */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 3a916a47d9c65..e2631b534ce37 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -15,6 +15,7 @@ import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.Metadata; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; import org.apache.kafka.clients.consumer.OffsetAndTimestamp; @@ -27,6 +28,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; @@ -72,7 +74,7 @@ /** * This class manage the fetching process with the brokers. */ -public class Fetcher { +public class Fetcher implements SubscriptionState.Listener { private static final Logger log = LoggerFactory.getLogger(Fetcher.class); @@ -93,6 +95,7 @@ public class Fetcher { private final Deserializer valueDeserializer; private PartitionRecords nextInLineRecords = null; + private ExceptionMetadata nextInLineExceptionMetadata = null; public Fetcher(ConsumerNetworkClient client, int minBytes, @@ -124,6 +127,30 @@ public Fetcher(ConsumerNetworkClient client, this.completedFetches = new ConcurrentLinkedQueue<>(); this.sensors = new FetchManagerMetrics(metrics, metricGrpPrefix); this.retryBackoffMs = retryBackoffMs; + + subscriptions.addListener(this); + } + + /** + * Represents data about an offset returned by a broker. + */ + private static class OffsetData { + /** + * The offset + */ + final long offset; + + /** + * The timestamp. + * + * Will be null if the broker does not support returning timestamps. + */ + final Long timestamp; + + OffsetData(long offset, Long timestamp) { + this.offset = offset; + this.timestamp = timestamp; + } } /** @@ -151,6 +178,7 @@ public int sendFetches() { final FetchRequest.Builder request = fetchEntry.getValue(); final Node fetchTarget = fetchEntry.getKey(); + log.debug("Sending fetch for partitions {} to broker {}", request.fetchData().keySet(), fetchTarget); client.send(fetchTarget, request) .addListener(new RequestFutureListener() { @Override @@ -172,7 +200,8 @@ public void onSuccess(ClientResponse resp) { TopicPartition partition = entry.getKey(); long fetchOffset = request.fetchData().get(partition).offset; FetchResponse.PartitionData fetchData = entry.getValue(); - completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator)); + completedFetches.add(new CompletedFetch(partition, fetchOffset, fetchData, metricAggregator, + request.version())); } sensors.fetchLatency.record(resp.requestLatencyMs()); @@ -181,7 +210,7 @@ public void onSuccess(ClientResponse resp) { @Override public void onFailure(RuntimeException e) { - log.debug("Fetch request to {} failed", fetchTarget, e); + log.debug("Fetch request to {} for partitions {} failed", fetchTarget, request.fetchData().keySet(), e); } }); } @@ -208,7 +237,7 @@ public void resetOffsetsIfNeeded(Set partitions) { public void updateFetchPositions(Set partitions) { // reset the fetch position to the committed position for (TopicPartition tp : partitions) { - if (!subscriptions.isAssigned(tp) || subscriptions.isFetchable(tp)) + if (!subscriptions.isAssigned(tp) || subscriptions.hasValidPosition(tp)) continue; if (subscriptions.isOffsetResetNeeded(tp)) { @@ -332,50 +361,39 @@ private RequestFuture sendMetadataRequest(MetadataRequest.Builde private void resetOffset(TopicPartition partition) { OffsetResetStrategy strategy = subscriptions.resetStrategy(partition); log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase(Locale.ROOT)); - long offset = listOffset(partition, strategy); + final long timestamp; + if (strategy == OffsetResetStrategy.EARLIEST) + timestamp = ListOffsetRequest.EARLIEST_TIMESTAMP; + else if (strategy == OffsetResetStrategy.LATEST) + timestamp = ListOffsetRequest.LATEST_TIMESTAMP; + else + throw new NoOffsetForPartitionException(partition); + Map offsetsByTimes = retrieveOffsetsByTimes( + Collections.singletonMap(partition, timestamp), Long.MAX_VALUE, false); + OffsetData offsetData = offsetsByTimes.get(partition); + if (offsetData == null) + throw new NoOffsetForPartitionException(partition); + long offset = offsetData.offset; // we might lose the assignment while fetching the offset, so check it is still active if (subscriptions.isAssigned(partition)) this.subscriptions.seek(partition, offset); } - private long listOffset(TopicPartition partition, OffsetResetStrategy strategy) { - final long timestamp; - switch (strategy) { - case EARLIEST: - timestamp = ListOffsetRequest.EARLIEST_TIMESTAMP; - break; - case LATEST: - timestamp = ListOffsetRequest.LATEST_TIMESTAMP; - break; - default: - throw new NoOffsetForPartitionException(partition); - } - while (true) { - RequestFuture> future = - sendListOffsetRequests(false, Collections.singletonMap(partition, timestamp)); - client.poll(future); - if (future.succeeded()) { - OffsetAndTimestamp offsetAndTimestamp = future.value().get(partition); - if (offsetAndTimestamp == null) - throw new NoOffsetForPartitionException(partition); - return offsetAndTimestamp.offset(); - } - if (!future.isRetriable()) - throw future.exception(); - if (future.exception() instanceof InvalidMetadataException) - client.awaitMetadataUpdate(); - else - time.sleep(retryBackoffMs); - } - } - - public Map getOffsetsByTimes(Map timestampsToSearch, long timeout) { - return retrieveOffsetsByTimes(timestampsToSearch, timeout, true); + Map offsetData = retrieveOffsetsByTimes(timestampsToSearch, timeout, true); + HashMap offsetsByTimes = new HashMap<>(offsetData.size()); + for (Map.Entry entry : offsetData.entrySet()) { + OffsetData data = entry.getValue(); + if (data == null) + offsetsByTimes.put(entry.getKey(), null); + else + offsetsByTimes.put(entry.getKey(), new OffsetAndTimestamp(data.offset, data.timestamp)); + } + return offsetsByTimes; } - private Map retrieveOffsetsByTimes( + private Map retrieveOffsetsByTimes( Map timestampsToSearch, long timeout, boolean requireTimestamps) { if (timestampsToSearch.isEmpty()) return Collections.emptyMap(); @@ -383,7 +401,7 @@ private Map retrieveOffsetsByTimes( long startMs = time.milliseconds(); long remaining = timeout; do { - RequestFuture> future = + RequestFuture> future = sendListOffsetRequests(requireTimestamps, timestampsToSearch); client.poll(future, remaining); @@ -427,9 +445,9 @@ private Map beginningOrEndOffset(Collection result = new HashMap<>(); - for (Map.Entry entry : + for (Map.Entry entry : retrieveOffsetsByTimes(timestampsToSearch, timeout, false).entrySet()) { - result.put(entry.getKey(), entry.getValue().offset()); + result.put(entry.getKey(), entry.getValue().offset); } return result; } @@ -444,19 +462,29 @@ private Map beginningOrEndOffset(Collection>> fetchedRecords() { + if (nextInLineExceptionMetadata != null) { + ExceptionMetadata exceptionMetadata = nextInLineExceptionMetadata; + nextInLineExceptionMetadata = null; + TopicPartition tp = exceptionMetadata.partition; + if (subscriptions.isFetchable(tp) && subscriptions.position(tp) == exceptionMetadata.fetchedOffset) + throw exceptionMetadata.exception; + } + Map>> drained = new HashMap<>(); int recordsRemaining = maxPollRecords; - while (recordsRemaining > 0) { if (nextInLineRecords == null || nextInLineRecords.isDrained()) { CompletedFetch completedFetch = completedFetches.poll(); - if (completedFetch == null) - break; - - nextInLineRecords = parseFetchedData(completedFetch); + if (completedFetch == null) break; + try { + nextInLineRecords = parseCompletedFetch(completedFetch); + } catch (KafkaException e) { + if (drained.isEmpty()) + throw e; + nextInLineExceptionMetadata = new ExceptionMetadata(completedFetch.partition, completedFetch.fetchedOffset, e); + } } else { TopicPartition partition = nextInLineRecords.partition; - List> records = drainRecords(nextInLineRecords, recordsRemaining); if (!records.isEmpty()) { List> currentRecords = drained.get(partition); @@ -476,13 +504,11 @@ public Map>> fetchedRecords() { } } + return drained; } private List> drainRecords(PartitionRecords partitionRecords, int maxRecords) { - if (partitionRecords.isDrained()) - return Collections.emptyList(); - if (!subscriptions.isAssigned(partitionRecords.partition)) { // this can happen when a rebalance happened before fetched records are returned to the consumer's poll call log.debug("Not returning fetched records for partition {} since it is no longer assigned", partitionRecords.partition); @@ -493,19 +519,19 @@ private List> drainRecords(PartitionRecords partition // this can happen when a partition is paused before fetched records are returned to the consumer's poll call log.debug("Not returning fetched records for assigned partition {} since it is no longer fetchable", partitionRecords.partition); } else if (partitionRecords.fetchOffset == position) { - // we are ensured to have at least one record since we already checked for emptiness List> partRecords = partitionRecords.drainRecords(maxRecords); - long nextOffset = partRecords.get(partRecords.size() - 1).offset() + 1; + if (!partRecords.isEmpty()) { + long nextOffset = partRecords.get(partRecords.size() - 1).offset() + 1; + log.trace("Returning fetched records at offset {} for assigned partition {} and update " + + "position to {}", position, partitionRecords.partition, nextOffset); - log.trace("Returning fetched records at offset {} for assigned partition {} and update " + - "position to {}", position, partitionRecords.partition, nextOffset); + subscriptions.position(partitionRecords.partition, nextOffset); + } - subscriptions.position(partitionRecords.partition, nextOffset); Long partitionLag = subscriptions.partitionLag(partitionRecords.partition); - if (partitionLag != null) { - this.sensors.recordsFetchLag.record(partitionLag); - this.sensors.recordPartitionFetchLag(partitionRecords.partition, partitionLag); - } + if (partitionLag != null) + this.sensors.recordPartitionLag(partitionRecords.partition, partitionLag); + return partRecords; } else { // these records aren't next in line based on the last consumed position, ignore them @@ -522,12 +548,12 @@ private List> drainRecords(PartitionRecords partition /** * Search the offsets by target times for the specified partitions. * - * @param requireTimestamps true if we should fail with an ObsoleteBrokerException if the broker does not support - * fetching precise timestamps for offsets + * @param requireTimestamps true if we should fail with an UnsupportedVersionException if the broker does + * not support fetching precise timestamps for offsets * @param timestampsToSearch the mapping between partitions and target time * @return A response which can be polled to obtain the corresponding timestamps and offsets. */ - private RequestFuture> sendListOffsetRequests( + private RequestFuture> sendListOffsetRequests( final boolean requireTimestamps, final Map timestampsToSearch) { // Group the partitions by node. @@ -553,14 +579,14 @@ private RequestFuture> sendListOffsetReq } } - final RequestFuture> listOffsetRequestsFuture = new RequestFuture<>(); - final Map fetchedTimestampOffsets = new HashMap<>(); + final RequestFuture> listOffsetRequestsFuture = new RequestFuture<>(); + final Map fetchedTimestampOffsets = new HashMap<>(); final AtomicInteger remainingResponses = new AtomicInteger(timestampsToSearchByNode.size()); for (Map.Entry> entry : timestampsToSearchByNode.entrySet()) { sendListOffsetRequest(entry.getKey(), entry.getValue(), requireTimestamps) - .addListener(new RequestFutureListener>() { + .addListener(new RequestFutureListener>() { @Override - public void onSuccess(Map value) { + public void onSuccess(Map value) { synchronized (listOffsetRequestsFuture) { fetchedTimestampOffsets.putAll(value); if (remainingResponses.decrementAndGet() == 0 && !listOffsetRequestsFuture.isDone()) @@ -589,7 +615,7 @@ public void onFailure(RuntimeException e) { * @param requireTimestamp True if we require a timestamp in the response. * @return A response which can be polled to obtain the corresponding timestamps and offsets. */ - private RequestFuture> sendListOffsetRequest(final Node node, + private RequestFuture> sendListOffsetRequest(final Node node, final Map timestampsToSearch, boolean requireTimestamp) { ListOffsetRequest.Builder builder = new ListOffsetRequest.Builder().setTargetTimes(timestampsToSearch); @@ -600,9 +626,9 @@ private RequestFuture> sendListOffsetReq log.trace("Sending ListOffsetRequest {} to broker {}", builder, node); return client.send(node, builder) - .compose(new RequestFutureAdapter>() { + .compose(new RequestFutureAdapter>() { @Override - public void onSuccess(ClientResponse response, RequestFuture> future) { + public void onSuccess(ClientResponse response, RequestFuture> future) { ListOffsetResponse lor = (ListOffsetResponse) response.responseBody(); log.trace("Received ListOffsetResponse {} from broker {}", lor, node); handleListOffsetResponse(timestampsToSearch, lor, future); @@ -619,8 +645,8 @@ public void onSuccess(ClientResponse response, RequestFuture timestampsToSearch, ListOffsetResponse listOffsetResponse, - RequestFuture> future) { - Map timestampOffsetMap = new HashMap<>(); + RequestFuture> future) { + Map timestampOffsetMap = new HashMap<>(); for (Map.Entry entry : timestampsToSearch.entrySet()) { TopicPartition topicPartition = entry.getKey(); ListOffsetResponse.PartitionData partitionData = listOffsetResponse.responseData().get(topicPartition); @@ -638,20 +664,19 @@ private void handleListOffsetResponse(Map timestampsToSear } else { offset = partitionData.offsets.get(0); } - log.debug("Handling v0 ListOffsetResponse response for {}. Fetched offset {}", + log.debug("Handling v0 ListOffsetResponse response for {}. Fetched offset {}", topicPartition, offset); if (offset != ListOffsetResponse.UNKNOWN_OFFSET) { - OffsetAndTimestamp offsetAndTimestamp = new OffsetAndTimestamp(offset, -1); - timestampOffsetMap.put(topicPartition, offsetAndTimestamp); + OffsetData offsetData = new OffsetData(offset, null); + timestampOffsetMap.put(topicPartition, offsetData); } } else { // Handle v1 and later response - log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}", + log.debug("Handling ListOffsetResponse response for {}. Fetched offset {}, timestamp {}", topicPartition, partitionData.offset, partitionData.timestamp); if (partitionData.offset != ListOffsetResponse.UNKNOWN_OFFSET) { - OffsetAndTimestamp offsetAndTimestamp = - new OffsetAndTimestamp(partitionData.offset, partitionData.timestamp); - timestampOffsetMap.put(topicPartition, offsetAndTimestamp); + OffsetData offsetData = new OffsetData(partitionData.offset, partitionData.timestamp); + timestampOffsetMap.put(topicPartition, offsetData); } } } else if (error == Errors.UNSUPPORTED_FOR_MESSAGE_FORMAT) { @@ -711,7 +736,7 @@ private Map createFetchRequests() { long position = this.subscriptions.position(partition); fetch.put(partition, new FetchRequest.PartitionData(position, this.fetchSize)); - log.trace("Added fetch request for partition {} at offset {}", partition, position); + log.trace("Added fetch request for partition {} at offset {} to node {}", partition, position, node); } else { log.trace("Skipping fetch for partition {} because there is an in-flight request to {}", partition, node); } @@ -731,7 +756,7 @@ private Map createFetchRequests() { /** * The callback for fetch completion */ - private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { + private PartitionRecords parseCompletedFetch(CompletedFetch completedFetch) { TopicPartition tp = completedFetch.partition; FetchResponse.PartitionData partition = completedFetch.partitionData; long fetchOffset = completedFetch.fetchedOffset; @@ -756,36 +781,43 @@ private PartitionRecords parseFetchedData(CompletedFetch completedFetch) { } List> parsed = new ArrayList<>(); + boolean skippedRecords = false; for (LogEntry logEntry : partition.records.deepEntries()) { // Skip the messages earlier than current position. if (logEntry.offset() >= position) { parsed.add(parseRecord(tp, logEntry)); bytes += logEntry.sizeInBytes(); - } + } else + skippedRecords = true; } recordsCount = parsed.size(); - if (!parsed.isEmpty()) { - log.trace("Adding fetched record for partition {} with offset {} to buffered record list", tp, position); - parsedRecords = new PartitionRecords<>(fetchOffset, tp, parsed); + log.trace("Adding fetched record for partition {} with offset {} to buffered record list", tp, position); + parsedRecords = new PartitionRecords<>(fetchOffset, tp, parsed); + + if (parsed.isEmpty() && !skippedRecords && (partition.records.sizeInBytes() > 0)) { + if (completedFetch.responseVersion < 3) { + // Implement the pre KIP-74 behavior of throwing a RecordTooLargeException. + Map recordTooLargePartitions = Collections.singletonMap(tp, fetchOffset); + throw new RecordTooLargeException("There are some messages at [Partition=Offset]: " + + recordTooLargePartitions + " whose size is larger than the fetch size " + this.fetchSize + + " and hence cannot be returned. Please considering upgrading your broker to 0.10.1.0 or " + + "newer to avoid this issue. Alternately, increase the fetch size on the client (using " + + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG + ")", + recordTooLargePartitions); + } else { + // This should not happen with brokers that support FetchRequest/Response V3 or higher (i.e. KIP-74) + throw new KafkaException("Failed to make progress reading messages at " + tp + "=" + + fetchOffset + ". Received a non-empty fetch response from the server, but no " + + "complete records were found."); + } } if (partition.highWatermark >= 0) { log.trace("Received {} records in fetch response for partition {} with offset {}", parsed.size(), tp, position); - Long partitionLag = subscriptions.partitionLag(tp); subscriptions.updateHighWatermark(tp, partition.highWatermark); - // If the partition lag is null, that means this is the first fetch response for this partition. - // We update the lag here to create the lag metric. This is to handle the case that there is no - // message consumed by the end user from this partition. If there are messages returned from the - // partition, the lag will be updated when those messages are consumed by the end user. - if (partitionLag == null) { - partitionLag = subscriptions.partitionLag(tp); - this.sensors.recordsFetchLag.record(partitionLag); - this.sensors.recordPartitionFetchLag(tp, partitionLag); - } } - } else if (error == Errors.NOT_LEADER_FOR_PARTITION) { log.debug("Error in fetch for partition {}: {}", tp, error.exceptionName()); this.metadata.requestUpdate(); @@ -860,20 +892,25 @@ private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logE } } + @Override + public void onAssignment(Set assignment) { + sensors.updatePartitionLagSensors(assignment); + } + private static class PartitionRecords { private long fetchOffset; private TopicPartition partition; private List> records; private int position = 0; - public PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { + private PartitionRecords(long fetchOffset, TopicPartition partition, List> records) { this.fetchOffset = fetchOffset; this.partition = partition; this.records = records; } private boolean isDrained() { - return records == null || position >= records.size(); + return records == null; } private void drain() { @@ -881,8 +918,10 @@ private void drain() { } private List> drainRecords(int n) { - if (isDrained()) + if (isDrained() || position >= records.size()) { + drain(); return Collections.emptyList(); + } // using a sublist avoids a potentially expensive list copy (depending on the size of the records // and the maximum we can return from poll). The cost is that we cannot mutate the returned sublist. @@ -902,15 +941,18 @@ private static class CompletedFetch { private final long fetchedOffset; private final FetchResponse.PartitionData partitionData; private final FetchResponseMetricAggregator metricAggregator; + private final short responseVersion; - public CompletedFetch(TopicPartition partition, - long fetchedOffset, - FetchResponse.PartitionData partitionData, - FetchResponseMetricAggregator metricAggregator) { + private CompletedFetch(TopicPartition partition, + long fetchedOffset, + FetchResponse.PartitionData partitionData, + FetchResponseMetricAggregator metricAggregator, + short responseVersion) { this.partition = partition; this.fetchedOffset = fetchedOffset; this.partitionData = partitionData; this.metricAggregator = metricAggregator; + this.responseVersion = responseVersion; } } @@ -974,16 +1016,17 @@ protected void increment(int bytes, int records) { } private static class FetchManagerMetrics { - public final Metrics metrics; - public final String metricGrpName; + private final Metrics metrics; + private final String metricGrpName; + private final Sensor bytesFetched; + private final Sensor recordsFetched; + private final Sensor fetchLatency; + private final Sensor recordsFetchLag; + private final Sensor fetchThrottleTimeSensor; - public final Sensor bytesFetched; - public final Sensor recordsFetched; - public final Sensor fetchLatency; - public final Sensor recordsFetchLag; - public final Sensor fetchThrottleTimeSensor; + private Set assignedPartitions; - public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix) { + private FetchManagerMetrics(Metrics metrics, String metricGrpPrefix) { this.metrics = metrics; this.metricGrpName = metricGrpPrefix + "-fetch-manager-metrics"; @@ -1032,7 +1075,7 @@ public FetchManagerMetrics(Metrics metrics, String metricGrpPrefix) { "The maximum throttle time in ms"), new Max()); } - public void recordTopicFetchMetrics(String topic, int bytes, int records) { + private void recordTopicFetchMetrics(String topic, int bytes, int records) { // record bytes fetched String name = "topic." + topic + ".bytes-fetched"; Sensor bytesFetched = this.metrics.getSensor(name); @@ -1075,8 +1118,20 @@ public void recordTopicFetchMetrics(String topic, int bytes, int records) { recordsFetched.record(records); } - public void recordPartitionFetchLag(TopicPartition tp, long lag) { - String name = tp + ".records-lag"; + private void updatePartitionLagSensors(Set assignedPartitions) { + if (this.assignedPartitions != null) { + for (TopicPartition tp : this.assignedPartitions) { + if (!assignedPartitions.contains(tp)) + metrics.removeSensor(partitionLagMetricName(tp)); + } + } + this.assignedPartitions = assignedPartitions; + } + + private void recordPartitionLag(TopicPartition tp, long lag) { + this.recordsFetchLag.record(lag); + + String name = partitionLagMetricName(tp); Sensor recordsLag = this.metrics.getSensor(name); if (recordsLag == null) { recordsLag = this.metrics.sensor(name); @@ -1091,6 +1146,22 @@ public void recordPartitionFetchLag(TopicPartition tp, long lag) { } recordsLag.record(lag); } + + private static String partitionLagMetricName(TopicPartition tp) { + return tp + ".records-lag"; + } + } + + private static class ExceptionMetadata { + private final TopicPartition partition; + private final long fetchedOffset; + private final KafkaException exception; + + private ExceptionMetadata(TopicPartition partition, long fetchedOffset, KafkaException exception) { + this.partition = partition; + this.fetchedOffset = fetchedOffset; + this.exception = exception; + } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index ad4f20b8b47ab..25995fb5c04c2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -17,7 +17,6 @@ import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.internals.PartitionStates; -import org.apache.kafka.common.metrics.Metrics; import java.util.ArrayList; import java.util.Collection; @@ -78,19 +77,19 @@ private enum SubscriptionType { /* Default offset reset strategy */ private final OffsetResetStrategy defaultResetStrategy; - /* Listener to be invoked when assignment changes */ + /* User-provided listener to be invoked when assignment changes */ private ConsumerRebalanceListener listener; - private final Metrics metrics; + /* Listeners provide a hook for internal state cleanup (e.g. metrics) on assignment changes */ + private List listeners = new ArrayList<>(); - public SubscriptionState(OffsetResetStrategy defaultResetStrategy, Metrics metrics) { + public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { this.defaultResetStrategy = defaultResetStrategy; this.subscription = Collections.emptySet(); this.assignment = new PartitionStates<>(); this.groupSubscription = new HashSet<>(); this.needsFetchCommittedOffsets = true; // initialize to true for the consumers to fetch offset upon starting up this.subscribedPattern = null; - this.metrics = metrics; this.subscriptionType = SubscriptionType.NONE; } @@ -160,7 +159,8 @@ public void assignFromUser(Set partitions) { setSubscriptionType(SubscriptionType.USER_ASSIGNED); if (!this.assignment.partitionSet().equals(partitions)) { - removeAllLagSensors(partitions); + fireOnAssignment(partitions); + Map partitionToState = new HashMap<>(); for (TopicPartition partition : partitions) { TopicPartitionState state = assignment.stateValue(partition); @@ -180,30 +180,24 @@ public void assignFromUser(Set partitions) { public void assignFromSubscribed(Collection assignments) { if (!this.partitionsAutoAssigned()) throw new IllegalArgumentException("Attempt to dynamically assign partitions while manual assignment in use"); - Set newAssignment = new HashSet<>(assignments); - removeAllLagSensors(newAssignment); - - for (TopicPartition tp : assignments) - if (!this.subscription.contains(tp.topic())) - throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic."); - // after rebalancing, we always reinitialize the assignment value - this.assignment.set(partitionToStateMap(assignments)); - this.needsFetchCommittedOffsets = true; - } + Map assignedPartitionStates = partitionToStateMap(assignments); + fireOnAssignment(assignedPartitionStates.keySet()); - private void removeAllLagSensors(Set preservedPartitions) { - for (TopicPartition tp : assignment.partitionSet()) { - if (!preservedPartitions.contains(tp)) - metrics.removeSensor(tp + ".records-lag"); + if (this.subscribedPattern != null) { + for (TopicPartition tp : assignments) { + if (!this.subscribedPattern.matcher(tp.topic()).matches()) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic regex pattern; subscription pattern is " + this.subscribedPattern); + } + } else { + for (TopicPartition tp : assignments) + if (!this.subscription.contains(tp.topic())) + throw new IllegalArgumentException("Assigned partition " + tp + " for non-subscribed topic; subscription is " + this.subscription); } - } - private Map partitionToStateMap(Collection assignments) { - Map map = new HashMap<>(assignments.size()); - for (TopicPartition tp : assignments) - map.put(tp, new TopicPartitionState()); - return map; + // after rebalancing, we always reinitialize the assignment value + this.assignment.set(assignedPartitionStates); + this.needsFetchCommittedOffsets = true; } public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { @@ -229,9 +223,10 @@ public void unsubscribe() { this.assignment.clear(); this.subscribedPattern = null; this.subscriptionType = SubscriptionType.NONE; + fireOnAssignment(Collections.emptySet()); } - public Pattern getSubscribedPattern() { + public Pattern subscribedPattern() { return this.subscribedPattern; } @@ -357,13 +352,17 @@ public OffsetResetStrategy resetStrategy(TopicPartition partition) { return assignedState(partition).resetStrategy; } - public boolean hasAllFetchPositions() { - for (TopicPartitionState state : assignment.partitionStateValues()) - if (!state.hasValidPosition()) + public boolean hasAllFetchPositions(Collection partitions) { + for (TopicPartition partition : partitions) + if (!hasValidPosition(partition)) return false; return true; } + public boolean hasAllFetchPositions() { + return hasAllFetchPositions(this.assignedPartitions()); + } + public Set missingFetchPositions() { Set missing = new HashSet<>(); for (PartitionStates.PartitionState state : assignment.partitionStates()) { @@ -385,6 +384,10 @@ public boolean isFetchable(TopicPartition tp) { return isAssigned(tp) && assignedState(tp).isFetchable(); } + public boolean hasValidPosition(TopicPartition tp) { + return isAssigned(tp) && assignedState(tp).hasValidPosition(); + } + public void pause(TopicPartition tp) { assignedState(tp).pause(); } @@ -401,6 +404,22 @@ public ConsumerRebalanceListener listener() { return listener; } + public void addListener(Listener listener) { + listeners.add(listener); + } + + public void fireOnAssignment(Set assignment) { + for (Listener listener : listeners) + listener.onAssignment(assignment); + } + + private static Map partitionToStateMap(Collection assignments) { + Map map = new HashMap<>(assignments.size()); + for (TopicPartition tp : assignments) + map.put(tp, new TopicPartitionState()); + return map; + } + private static class TopicPartitionState { private Long position; // last consumed position private Long highWatermark; // the high watermark from last fetch @@ -458,4 +477,14 @@ private boolean isFetchable() { } + public interface Listener { + /** + * Fired after a new assignment is received (after a group rebalance or when the user manually changes the + * assignment). + * + * @param assignment The topic partitions assigned to the consumer + */ + void onAssignment(Set assignment); + } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index c604daa86a5c7..6f1a3b64a7b8a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -124,6 +124,10 @@ * The key.serializer and value.serializer instruct how to turn the key and value objects the user provides with * their ProducerRecord into bytes. You can use the included {@link org.apache.kafka.common.serialization.ByteArraySerializer} or * {@link org.apache.kafka.common.serialization.StringSerializer} for simple string or byte types. + *

+ * This client can communicate with brokers that are version 0.10.0 or newer. Older or newer brokers may not support + * certain client features. You will receive an UnsupportedVersionException when invoking an API that is not available + * with the running broker verion. */ public class KafkaProducer implements Producer { @@ -297,7 +301,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial time); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), time.milliseconds()); ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values()); NetworkClient client = new NetworkClient( new Selector(config.getLong(ProducerConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), this.metrics, time, "producer", channelBuilder), diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java index 109b0caed4963..165437de1085e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java @@ -116,11 +116,11 @@ public synchronized Future send(ProducerRecord record, Cal int partition = 0; if (this.cluster.partitionsForTopic(record.topic()) != null) partition = partition(record, this.cluster); - ProduceRequestResult result = new ProduceRequestResult(); - FutureRecordMetadata future = new FutureRecordMetadata(result, 0, Record.NO_TIMESTAMP, 0, 0, 0); TopicPartition topicPartition = new TopicPartition(record.topic(), partition); + ProduceRequestResult result = new ProduceRequestResult(topicPartition); + FutureRecordMetadata future = new FutureRecordMetadata(result, 0, Record.NO_TIMESTAMP, 0, 0, 0); long offset = nextOffset(topicPartition); - Completion completion = new Completion(topicPartition, offset, + Completion completion = new Completion(offset, new RecordMetadata(topicPartition, 0, offset, Record.NO_TIMESTAMP, 0, 0, 0), result, callback); this.sent.add(record); @@ -233,10 +233,8 @@ private static class Completion { private final RecordMetadata metadata; private final ProduceRequestResult result; private final Callback callback; - private final TopicPartition topicPartition; - public Completion(TopicPartition topicPartition, - long offset, + public Completion(long offset, RecordMetadata metadata, ProduceRequestResult result, Callback callback) { @@ -244,17 +242,17 @@ public Completion(TopicPartition topicPartition, this.offset = offset; this.result = result; this.callback = callback; - this.topicPartition = topicPartition; } public void complete(RuntimeException e) { - result.done(topicPartition, e == null ? offset : -1L, e); + result.set(e == null ? offset : -1L, Record.NO_TIMESTAMP, e); if (callback != null) { if (e == null) callback.onCompletion(metadata, null); else callback.onCompletion(null, e); } + result.done(); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java index d5995a3468c10..1cde13d1b2d6e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java @@ -26,16 +26,16 @@ public final class FutureRecordMetadata implements Future { private final ProduceRequestResult result; private final long relativeOffset; - private final long timestamp; + private final long createTimestamp; private final long checksum; private final int serializedKeySize; private final int serializedValueSize; - public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset, long timestamp, + public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset, long createTimestamp, long checksum, int serializedKeySize, int serializedValueSize) { this.result = result; this.relativeOffset = relativeOffset; - this.timestamp = timestamp; + this.createTimestamp = createTimestamp; this.checksum = checksum; this.serializedKeySize = serializedKeySize; this.serializedValueSize = serializedValueSize; @@ -46,6 +46,11 @@ public boolean cancel(boolean interrupt) { return false; } + @Override + public boolean isCancelled() { + return false; + } + @Override public RecordMetadata get() throws InterruptedException, ExecutionException { this.result.await(); @@ -69,32 +74,11 @@ RecordMetadata valueOrError() throws ExecutionException { RecordMetadata value() { return new RecordMetadata(result.topicPartition(), this.result.baseOffset(), this.relativeOffset, - this.timestamp, this.checksum, this.serializedKeySize, this.serializedValueSize); - } - - public long relativeOffset() { - return this.relativeOffset; - } - - public long timestamp() { - return this.timestamp; + timestamp(), this.checksum, this.serializedKeySize, this.serializedValueSize); } - public long checksum() { - return this.checksum; - } - - public int serializedKeySize() { - return this.serializedKeySize; - } - - public int serializedValueSize() { - return this.serializedValueSize; - } - - @Override - public boolean isCancelled() { - return false; + private long timestamp() { + return result.hasLogAppendTime() ? result.logAppendTime() : createTimestamp; } @Override diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java index 8e5855d9f41e2..a98b28e52728c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java @@ -19,7 +19,9 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; +import org.apache.kafka.clients.producer.RecordMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; /** @@ -30,23 +32,40 @@ public final class ProduceRequestResult { private final CountDownLatch latch = new CountDownLatch(1); - private volatile TopicPartition topicPartition; - private volatile long baseOffset = -1L; + private final TopicPartition topicPartition; + + private volatile Long baseOffset = null; + private volatile long logAppendTime = Record.NO_TIMESTAMP; private volatile RuntimeException error; - public ProduceRequestResult() { + /** + * Create an instance of this class. + * + * @param topicPartition The topic and partition to which this record set was sent was sent + */ + public ProduceRequestResult(TopicPartition topicPartition) { + this.topicPartition = topicPartition; } /** - * Mark this request as complete and unblock any threads waiting on its completion. - * @param topicPartition The topic and partition to which this record set was sent was sent + * Set the result of the produce request. + * * @param baseOffset The base offset assigned to the record - * @param error The error that occurred if there was one, or null. + * @param logAppendTime The log append time or -1 if CreateTime is being used + * @param error The error that occurred if there was one, or null */ - public void done(TopicPartition topicPartition, long baseOffset, RuntimeException error) { - this.topicPartition = topicPartition; + public void set(long baseOffset, long logAppendTime, RuntimeException error) { this.baseOffset = baseOffset; + this.logAppendTime = logAppendTime; this.error = error; + } + + /** + * Mark this request as complete and unblock any threads waiting on its completion. + */ + public void done() { + if (baseOffset == null) + throw new IllegalStateException("The method `set` must be invoked before this method."); this.latch.countDown(); } @@ -74,6 +93,20 @@ public long baseOffset() { return baseOffset; } + /** + * Return true if log append time is being used for this topic + */ + public boolean hasLogAppendTime() { + return logAppendTime != Record.NO_TIMESTAMP; + } + + /** + * The log append time or -1 if CreateTime is being used + */ + public long logAppendTime() { + return logAppendTime; + } + /** * The error thrown (generally on the server) while processing this request */ diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 06d39ecc68cab..d3ae89e30881f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -243,12 +243,14 @@ public List abortExpiredBatches(int requestTimeout, long now) { while (batchIterator.hasNext()) { RecordBatch batch = batchIterator.next(); boolean isFull = batch != lastBatch || batch.isFull(); - // check if the batch is expired + // Check if the batch has expired. Expired batches are closed by maybeExpire, but callbacks + // are invoked after completing the iterations, since sends invoked from callbacks + // may append more batches to the deque being iterated. The batch is deallocated after + // callbacks are invoked. if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) { expiredBatches.add(batch); count++; batchIterator.remove(); - deallocate(batch); } else { // Stop at the first batch that has not expired. break; @@ -257,8 +259,13 @@ public List abortExpiredBatches(int requestTimeout, long now) { } } } - if (!expiredBatches.isEmpty()) + if (!expiredBatches.isEmpty()) { log.trace("Expired {} batches in accumulator", count); + for (RecordBatch batch : expiredBatches) { + batch.expirationDone(); + deallocate(batch); + } + } return expiredBatches; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index 68b27d3fa14a4..6346fe9c9f033 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -25,6 +25,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; /** * A batch of records that is or will be sent. @@ -35,28 +36,31 @@ public final class RecordBatch { private static final Logger log = LoggerFactory.getLogger(RecordBatch.class); - public int recordCount = 0; - public int maxRecordSize = 0; - public volatile int attempts = 0; - public final long createdMs; - public long drainedMs; - public long lastAttemptMs; - public final TopicPartition topicPartition; - public final ProduceRequestResult produceFuture; - public long lastAppendTime; - private final List thunks; - private boolean retry; + final long createdMs; + final TopicPartition topicPartition; + final ProduceRequestResult produceFuture; + + private final List thunks = new ArrayList<>(); private final MemoryRecordsBuilder recordsBuilder; + volatile int attempts; + int recordCount; + int maxRecordSize; + long drainedMs; + long lastAttemptMs; + long lastAppendTime; + private String expiryErrorMessage; + private AtomicBoolean completed; + private boolean retry; + public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) { this.createdMs = now; this.lastAttemptMs = now; this.recordsBuilder = recordsBuilder; this.topicPartition = tp; - this.produceFuture = new ProduceRequestResult(); - this.thunks = new ArrayList<>(); this.lastAppendTime = createdMs; - this.retry = false; + this.produceFuture = new ProduceRequestResult(topicPartition); + this.completed = new AtomicBoolean(); } /** @@ -83,36 +87,37 @@ public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, } /** - * Complete the request + * Complete the request. * * @param baseOffset The base offset of the messages assigned by the server - * @param timestamp The timestamp returned by the broker. + * @param logAppendTime The log append time or -1 if CreateTime is being used * @param exception The exception that occurred (or null if the request was successful) */ - public void done(long baseOffset, long timestamp, RuntimeException exception) { + public void done(long baseOffset, long logAppendTime, RuntimeException exception) { log.trace("Produced messages to topic-partition {} with base offset offset {} and error: {}.", - topicPartition, - baseOffset, - exception); + topicPartition, baseOffset, exception); + + if (completed.getAndSet(true)) + throw new IllegalStateException("Batch has already been completed"); + + // Set the future before invoking the callbacks as we rely on its state for the `onCompletion` call + produceFuture.set(baseOffset, logAppendTime, exception); + // execute callbacks for (Thunk thunk : thunks) { try { if (exception == null) { - // If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used. - RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(), - timestamp == Record.NO_TIMESTAMP ? thunk.future.timestamp() : timestamp, - thunk.future.checksum(), - thunk.future.serializedKeySize(), - thunk.future.serializedValueSize()); + RecordMetadata metadata = thunk.future.value(); thunk.callback.onCompletion(metadata, null); } else { thunk.callback.onCompletion(null, exception); } } catch (Exception e) { - log.error("Error executing user-provided callback on message for topic-partition {}:", topicPartition, e); + log.error("Error executing user-provided callback on message for topic-partition '{}'", topicPartition, e); } } - this.produceFuture.done(topicPartition, baseOffset, exception); + + produceFuture.done(); } /** @@ -139,35 +144,40 @@ public String toString() { *

  • the batch is not in retry AND request timeout has elapsed after it is ready (full or linger.ms has reached). *
  • the batch is in retry AND request timeout has elapsed after the backoff period ended. * + * This methods closes this batch and sets {@code expiryErrorMessage} if the batch has timed out. + * {@link #expirationDone()} must be invoked to complete the produce future and invoke callbacks. */ public boolean maybeExpire(int requestTimeoutMs, long retryBackoffMs, long now, long lingerMs, boolean isFull) { - boolean expire = false; - String errorMessage = null; - - if (!this.inRetry() && isFull && requestTimeoutMs < (now - this.lastAppendTime)) { - expire = true; - errorMessage = (now - this.lastAppendTime) + " ms has passed since last append"; - } else if (!this.inRetry() && requestTimeoutMs < (now - (this.createdMs + lingerMs))) { - expire = true; - errorMessage = (now - (this.createdMs + lingerMs)) + " ms has passed since batch creation plus linger time"; - } else if (this.inRetry() && requestTimeoutMs < (now - (this.lastAttemptMs + retryBackoffMs))) { - expire = true; - errorMessage = (now - (this.lastAttemptMs + retryBackoffMs)) + " ms has passed since last attempt plus backoff time"; - } - if (expire) { + if (!this.inRetry() && isFull && requestTimeoutMs < (now - this.lastAppendTime)) + expiryErrorMessage = (now - this.lastAppendTime) + " ms has passed since last append"; + else if (!this.inRetry() && requestTimeoutMs < (now - (this.createdMs + lingerMs))) + expiryErrorMessage = (now - (this.createdMs + lingerMs)) + " ms has passed since batch creation plus linger time"; + else if (this.inRetry() && requestTimeoutMs < (now - (this.lastAttemptMs + retryBackoffMs))) + expiryErrorMessage = (now - (this.lastAttemptMs + retryBackoffMs)) + " ms has passed since last attempt plus backoff time"; + + boolean expired = expiryErrorMessage != null; + if (expired) close(); - this.done(-1L, Record.NO_TIMESTAMP, - new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage)); - } + return expired; + } - return expire; + /** + * Completes the produce future with timeout exception and invokes callbacks. + * This method should be invoked only if {@link #maybeExpire(int, long, long, long, boolean)} + * returned true. + */ + void expirationDone() { + if (expiryErrorMessage == null) + throw new IllegalStateException("Batch has not expired"); + this.done(-1L, Record.NO_TIMESTAMP, + new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + ": " + expiryErrorMessage)); } /** * Returns if the batch is been retried for sending to kafka */ - public boolean inRetry() { + private boolean inRetry() { return this.retry; } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index dc299362fe4dd..5483c4a09e717 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -34,7 +34,6 @@ import org.apache.kafka.common.metrics.stats.Rate; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.Record; import org.apache.kafka.common.requests.ProduceRequest; import org.apache.kafka.common.requests.ProduceResponse; import org.apache.kafka.common.utils.Time; @@ -253,12 +252,12 @@ private void handleProduceResponse(ClientResponse response, Map entry : produceResponse.responses().entrySet()) { TopicPartition tp = entry.getKey(); ProduceResponse.PartitionResponse partResp = entry.getValue(); - Errors error = Errors.forCode(partResp.errorCode); RecordBatch batch = batches.get(tp); - completeBatch(batch, error, partResp.baseOffset, partResp.timestamp, correlationId, now); + completeBatch(batch, partResp, correlationId, now); } this.sensors.recordLatency(response.destination(), response.requestLatencyMs()); this.sensors.recordThrottleTime(produceResponse.getThrottleTime()); } else { // this is the acks = 0 case, just complete all requests - for (RecordBatch batch : batches.values()) - completeBatch(batch, Errors.NONE, -1L, Record.NO_TIMESTAMP, correlationId, now); + for (RecordBatch batch : batches.values()) { + completeBatch(batch, new ProduceResponse.PartitionResponse(Errors.NONE), correlationId, now); + } } } } @@ -285,13 +284,13 @@ private void handleProduceResponse(ClientResponse response, Map T getConfiguredInstance(String key, Class t) { * @return The list of configured instances */ public List getConfiguredInstances(String key, Class t) { + return getConfiguredInstances(key, t, Collections.EMPTY_MAP); + } + + /** + * Get a list of configured instances of the given class specified by the given configuration key. The configuration + * may specify either null or an empty string to indicate no configured instances. In both cases, this method + * returns an empty list to indicate no configured instances. + * @param key The configuration key for the class + * @param t The interface the class should implement + * @param configOverrides Configuration overrides to use. + * @return The list of configured instances + */ + public List getConfiguredInstances(String key, Class t, Map configOverrides) { List klasses = getList(key); List objects = new ArrayList(); if (klasses == null) return objects; + Map configPairs = originals(); + configPairs.putAll(configOverrides); for (Object klass : klasses) { Object o; if (klass instanceof String) { @@ -236,7 +251,7 @@ public List getConfiguredInstances(String key, Class t) { if (!t.isInstance(o)) throw new KafkaException(klass + " is not an instance of " + t.getName()); if (o instanceof Configurable) - ((Configurable) o).configure(originals()); + ((Configurable) o).configure(configPairs); objects.add(t.cast(o)); } return objects; 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 89feb9aba5ed6..5257f6ef8ec9f 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 @@ -638,7 +638,7 @@ else if (value instanceof String) } else if (value instanceof String) { return Integer.parseInt(trimmed); } else { - throw new ConfigException(name, value, "Expected value to be an number."); + throw new ConfigException(name, value, "Expected value to be a 32-bit integer, but it was a " + value.getClass().getName()); } case SHORT: if (value instanceof Short) { @@ -646,7 +646,7 @@ else if (value instanceof String) } else if (value instanceof String) { return Short.parseShort(trimmed); } else { - throw new ConfigException(name, value, "Expected value to be an number."); + throw new ConfigException(name, value, "Expected value to be a 16-bit integer (short), but it was a " + value.getClass().getName()); } case LONG: if (value instanceof Integer) @@ -656,14 +656,14 @@ else if (value instanceof String) else if (value instanceof String) return Long.parseLong(trimmed); else - throw new ConfigException(name, value, "Expected value to be an number."); + throw new ConfigException(name, value, "Expected value to be a 64-bit integer (long), but it was a " + value.getClass().getName()); case DOUBLE: if (value instanceof Number) return ((Number) value).doubleValue(); else if (value instanceof String) return Double.parseDouble(trimmed); else - throw new ConfigException(name, value, "Expected value to be an number."); + throw new ConfigException(name, value, "Expected value to be a double, but it was a " + value.getClass().getName()); case LIST: if (value instanceof List) return (List) value; diff --git a/clients/src/main/java/org/apache/kafka/common/errors/ObsoleteBrokerException.java b/clients/src/main/java/org/apache/kafka/common/errors/ObsoleteBrokerException.java deleted file mode 100644 index 18f822d7750bf..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/errors/ObsoleteBrokerException.java +++ /dev/null @@ -1,31 +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.errors; - -/** - * Indicates that a request cannot be completed because an obsolete broker - * does not support the required functionality. - */ -public class ObsoleteBrokerException extends ApiException { - - private static final long serialVersionUID = 1L; - - public ObsoleteBrokerException(String message, Throwable cause) { - super(message, cause); - } - - public ObsoleteBrokerException(String message) { - super(message); - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java index 7923444c2a8f6..393a6df4bf6dd 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/PolicyViolationException.java @@ -17,6 +17,9 @@ package org.apache.kafka.common.errors; +/** + * Exception thrown if a create topics request does not satisfy the configured policy for a topic. + */ public class PolicyViolationException extends ApiException { public PolicyViolationException(String message) { diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java index 7b303facb5962..512c18e74aca2 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java @@ -443,6 +443,10 @@ public Map metrics() { return this.metrics; } + public List reporters() { + return this.reporters; + } + public KafkaMetric metric(MetricName metricName) { return this.metrics.get(metricName); } diff --git a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java index 159c30190e985..307eadf8db6c9 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java +++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java @@ -33,8 +33,8 @@ public ByteBufferReceive(String source, ByteBuffer... buffers) { super(); this.source = source; this.buffers = buffers; - for (int i = 0; i < buffers.length; i++) - remaining += buffers[i].remaining(); + for (ByteBuffer buffer : buffers) + remaining += buffer.remaining(); } @Override 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 239780ca72077..614f12f577dda 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 @@ -89,7 +89,7 @@ private static String toHtml() { for (ApiKeys key : ApiKeys.values()) { b.append("\n"); b.append(""); - b.append(key.name); + b.append("" + key.name + ""); b.append(""); b.append(""); b.append(key.id); 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 f30f889054cec..e7689e2e11908 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 @@ -166,7 +166,7 @@ public enum Errors { " the message was sent to an incompatible broker. See the broker logs for more details.")), UNSUPPORTED_FOR_MESSAGE_FORMAT(43, new UnsupportedForMessageFormatException("The message format version on the broker does not support the request.")), - POLICY_VIOLATION(44, new PolicyViolationException("Request parameters do not satisfy the system policy.")); + POLICY_VIOLATION(44, new PolicyViolationException("Request parameters do not satisfy the configured policy.")); private static final Logger log = LoggerFactory.getLogger(Errors.class); 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 e76fc047d1ae8..032e00e0a158f 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 @@ -193,7 +193,7 @@ public class Protocol { INT16), new Field("base_offset", INT64), - new Field("timestamp", + new Field("log_append_time", INT64, "The timestamp returned by broker after appending the messages. " + "If CreateTime is used for the topic, the timestamp will be -1. " + @@ -1176,10 +1176,11 @@ public static String toHtml() { for (ApiKeys key : ApiKeys.values()) { // Key b.append("
    "); + b.append(""); b.append(key.name); b.append(" API (Key: "); b.append(key.id); - b.append("):
    \n\n"); + b.append("):\n\n"); // Requests b.append("Requests:
    \n"); Schema[] requests = REQUESTS[key.id]; diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java index ae393b03c0d62..dd9cc84dfb253 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.errors.CorruptRecordException; +import org.apache.kafka.common.utils.Utils; import java.io.IOException; import java.nio.ByteBuffer; @@ -56,9 +57,7 @@ public FileChannelLogEntry nextEntry() throws IOException { return null; logHeaderBuffer.rewind(); - channel.read(logHeaderBuffer, position); - if (logHeaderBuffer.hasRemaining()) - return null; + Utils.readFullyOrFail(channel, logHeaderBuffer, position, "log header"); logHeaderBuffer.rewind(); long offset = logHeaderBuffer.getLong(); @@ -117,9 +116,7 @@ public byte magic() { try { byte[] magic = new byte[1]; ByteBuffer buf = ByteBuffer.wrap(magic); - channel.read(buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET); - if (buf.hasRemaining()) - throw new KafkaException("Failed to read magic byte from FileChannel " + channel); + Utils.readFullyOrFail(channel, buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET, "magic byte"); return magic[0]; } catch (IOException e) { throw new KafkaException(e); @@ -136,9 +133,7 @@ private Record loadRecord() throws IOException { return record; ByteBuffer recordBuffer = ByteBuffer.allocate(recordSize); - channel.read(recordBuffer, position + Records.LOG_OVERHEAD); - if (recordBuffer.hasRemaining()) - throw new IOException("Failed to read full record from channel " + channel); + Utils.readFullyOrFail(channel, recordBuffer, position + Records.LOG_OVERHEAD, "full record"); recordBuffer.rewind(); record = new Record(recordBuffer); diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java index 8a33dcaaef828..960b7168ff70f 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java @@ -108,14 +108,17 @@ public FileChannel channel() { } /** - * Read log entries into a given buffer. + * Read log entries into the given buffer until there are no bytes remaining in the buffer or the end of the file + * is reached. + * * @param buffer The buffer to write the entries to * @param position Position in the buffer to read from * @return The same buffer - * @throws IOException + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions */ public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException { - channel.read(buffer, position + this.start); + Utils.readFully(channel, buffer, position + this.start); buffer.flip(); return buffer; } diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java index 69e9003ff7f6e..02bfc24d73fbb 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java @@ -105,6 +105,20 @@ public Constructor get() throws ClassNotFoundException, NoSuchMethodException { private MemoryRecords builtRecords; + /** + * Construct a new builder. + * + * @param buffer The underlying buffer to use (note that this class will allocate a new buffer if necessary + * to fit the records appended) + * @param magic The magic value to use + * @param compressionType The compression codec to use + * @param timestampType The desired timestamp type. For magic > 0, this cannot be {@link TimestampType#NO_TIMESTAMP_TYPE}. + * @param baseOffset The initial offset to use for + * @param logAppendTime The log append time of this record set. Can be set to NO_TIMESTAMP if CREATE_TIME is used. + * @param writeLimit The desired limit on the total bytes for this record set (note that this can be exceeded + * when compression is used since size estimates are rough, and in the case that the first + * record added exceeds the size). + */ public MemoryRecordsBuilder(ByteBuffer buffer, byte magic, CompressionType compressionType, @@ -373,7 +387,9 @@ public boolean isClosed() { } public boolean isFull() { - return isClosed() || this.writeLimit <= estimatedBytesWritten(); + // note that the write limit is respected only after the first record is added which ensures we can always + // create non-empty batches (this is used to disable batching when the producer's batch size is set to 0). + return isClosed() || (this.numRecords > 0 && this.writeLimit <= estimatedBytesWritten()); } public int sizeInBytes() { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java index 542c6af19943d..0a4bce01394ac 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/CreateTopicsRequest.java @@ -16,7 +16,7 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.errors.ObsoleteBrokerException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -115,7 +115,8 @@ public Builder(Map topics, int timeout, boolean validateOn @Override public CreateTopicsRequest build() { if (validateOnly && version() == 0) - throw new ObsoleteBrokerException("validateOnly is not supported in version 0 of CreateTopicsRequest"); + throw new UnsupportedVersionException("validateOnly is not supported in version 0 of " + + "CreateTopicsRequest"); return new CreateTopicsRequest(topics, timeout, validateOnly, version()); } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index 4768a205e8f56..5700e9e184996 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -142,6 +142,7 @@ public String toString() { append(", replicaId=").append(replicaId). append(", maxWait=").append(maxWait). append(", minBytes=").append(minBytes). + append(", maxBytes=").append(maxBytes). append(", fetchData=").append(Utils.mkString(fetchData)). append(")"); return bld.toString(); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java index 66b01a0d39dbf..965b2073cfbf6 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchResponse.java @@ -82,6 +82,12 @@ public PartitionData(short errorCode, long highWatermark, Records records) { this.highWatermark = highWatermark; this.records = records; } + + @Override + public String toString() { + return "(errorCode=" + errorCode + ", highWaterMark=" + highWatermark + + ", records=" + records + ")"; + } } /** diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java index 33613831ffdcd..79251ed794317 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ListOffsetRequest.java @@ -17,7 +17,6 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.ObsoleteBrokerException; import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -88,7 +87,8 @@ public Builder setTargetTimes(Map partitionTimestamps) { public ListOffsetRequest build() { short version = version(); if (version < minVersion) { - throw new ObsoleteBrokerException("The broker is too old to send this request."); + throw new UnsupportedVersionException("Cannot create a v" + version + " ListOffsetRequest because " + + "we require features supported only in " + minVersion + " or later."); } if (version == 0) { if (offsetData == null) { @@ -106,7 +106,7 @@ public ListOffsetRequest build() { } } else { if (offsetData != null) { - throw new UnsupportedVersionException("Cannot create a v" + version + " ListOffsetRequest with v0 " + + throw new RuntimeException("Cannot create a v" + version + " ListOffsetRequest with v0 " + "PartitionData."); } else if (partitionTimestamps == null) { throw new RuntimeException("Must set partitionTimestamps when creating a v" + diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index a8baee52f3a08..7209db5304526 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -15,6 +15,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.errors.InvalidMetadataException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -261,6 +262,29 @@ public Set topicsByError(Errors error) { return errorTopics; } + /** + * Returns the set of topics with an error indicating invalid metadata + * and topics with any partition whose error indicates invalid metadata. + * This includes all non-existent topics specified in the metadata request + * and any topic returned with one or more partitions whose leader is not known. + */ + public Set unavailableTopics() { + Set invalidMetadataTopics = new HashSet<>(); + for (TopicMetadata topicMetadata : this.topicMetadata) { + if (topicMetadata.error.exception() instanceof InvalidMetadataException) + invalidMetadataTopics.add(topicMetadata.topic); + else { + for (PartitionMetadata partitionMetadata : topicMetadata.partitionMetadata) { + if (partitionMetadata.error.exception() instanceof InvalidMetadataException) { + invalidMetadataTopics.add(topicMetadata.topic); + break; + } + } + } + } + return invalidMetadataTopics; + } + /** * Get a snapshot of the cluster metadata from this response * @return the cluster snapshot 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 0ff49bed09c2d..43ddf88d02b4f 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 @@ -13,7 +13,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.ObsoleteBrokerException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -23,6 +23,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import java.util.Map; @@ -59,7 +60,8 @@ public boolean isAllTopicPartitions() { @Override public OffsetFetchRequest build() { if (isAllTopicPartitions() && version() < 2) - throw new ObsoleteBrokerException("The broker is too old to send this request."); + throw new UnsupportedVersionException("The broker only supports OffsetFetchRequest " + + "v" + version() + ", but we need v2 or newer to request all topic partitions."); return new OffsetFetchRequest(groupId, partitions, version()); } @@ -78,7 +80,7 @@ public String toString() { private final List partitions; public static OffsetFetchRequest forAllPartitions(String groupId) { - return new OffsetFetchRequest.Builder(groupId, (List) null).setVersion((short) 2).build(); + return new OffsetFetchRequest.Builder(groupId, null).setVersion((short) 2).build(); } // v0, v1, and v2 have the same fields. @@ -131,20 +133,35 @@ public OffsetFetchRequest(Struct struct, short versionId) { groupId = struct.getString(GROUP_ID_KEY_NAME); } - @Override - public AbstractResponse getErrorResponse(Throwable e) { + public OffsetFetchResponse getErrorResponse(Errors error) { short versionId = version(); + + Map responsePartitions = new HashMap<>(); + if (versionId < 2) { + for (TopicPartition partition : this.partitions) { + responsePartitions.put(partition, new OffsetFetchResponse.PartitionData( + OffsetFetchResponse.INVALID_OFFSET, + OffsetFetchResponse.NO_METADATA, + error)); + } + } + switch (versionId) { case 0: case 1: case 2: - return new OffsetFetchResponse(Errors.forException(e), partitions, versionId); + return new OffsetFetchResponse(error, responsePartitions, versionId); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.OFFSET_FETCH.id))); } } + @Override + public OffsetFetchResponse getErrorResponse(Throwable e) { + return getErrorResponse(Errors.forException(e)); + } + public String groupId() { return groupId; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java index 0095f38ec10f5..9c1415538c54f 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetFetchResponse.java @@ -46,6 +46,8 @@ public class OffsetFetchResponse extends AbstractResponse { public static final long INVALID_OFFSET = -1L; public static final String NO_METADATA = ""; + public static final PartitionData UNKNOWN_PARTITION = new PartitionData(INVALID_OFFSET, NO_METADATA, + Errors.UNKNOWN_TOPIC_OR_PARTITION); /** * Possible error codes: @@ -59,7 +61,7 @@ public class OffsetFetchResponse extends AbstractResponse { * - GROUP_AUTHORIZATION_FAILED (30) */ - public static final List PARTITION_ERRORS = Arrays.asList( + private static final List PARTITION_ERRORS = Arrays.asList( Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.TOPIC_AUTHORIZATION_FAILED); @@ -82,14 +84,30 @@ public boolean hasError() { } } - private List getTopicArray(Map responseData) { - Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + /** + * Constructor for the latest version. + * @param error Potential coordinator or group level error code + * @param responseData Fetched offset information grouped by topic-partition + */ + public OffsetFetchResponse(Errors error, Map responseData) { + this(error, responseData, CURRENT_VERSION); + } - List topicArray = new ArrayList(); + /** + * Unified constructor for all versions. + * @param error Potential coordinator or group level error code (for api version 2 and later) + * @param responseData Fetched offset information grouped by topic-partition + * @param version The request API version + */ + public OffsetFetchResponse(Errors error, Map responseData, int version) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); + + Map> topicsData = CollectionUtils.groupDataByTopic(responseData); + List topicArray = new ArrayList<>(); for (Map.Entry> entries : topicsData.entrySet()) { Struct topicData = this.struct.instance(RESPONSES_KEY_NAME); topicData.set(TOPIC_KEY_NAME, entries.getKey()); - List partitionArray = new ArrayList(); + List partitionArray = new ArrayList<>(); for (Map.Entry partitionEntry : entries.getValue().entrySet()) { PartitionData fetchPartitionData = partitionEntry.getValue(); Struct partitionData = topicData.instance(PARTITIONS_KEY_NAME); @@ -103,66 +121,17 @@ private List getTopicArray(Map responseDa topicArray.add(topicData); } - return topicArray; - } - - /** - * Unified constructor - * @param responseData Fetched offset information grouped by topic-partition - * @param topLevelErrorCode Potential coordinator or group level error code (for api version 2 and later) - * @param version The request API version - */ - public OffsetFetchResponse(Errors topLevelError, Map responseData, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - this.struct.set(RESPONSES_KEY_NAME, getTopicArray(responseData).toArray()); + this.struct.set(RESPONSES_KEY_NAME, topicArray.toArray()); this.responseData = responseData; - this.error = topLevelError; + this.error = error; if (version > 1) this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); } - /** - * Unified constructor (used only if there are errors in the response) - * @param partitions partitions to be included in the response - * @param topLevelErrorCode The error code to be reported in the response - * @param version The request API version - */ - public OffsetFetchResponse(Errors topLevelError, List partitions, int version) { - super(new Struct(ProtoUtils.responseSchema(ApiKeys.OFFSET_FETCH.id, version))); - - assert topLevelError != Errors.NONE; - this.responseData = new HashMap<>(); - this.error = topLevelError; - if (version < 2) { - for (TopicPartition partition : partitions) { - this.responseData.put(partition, new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, - OffsetFetchResponse.NO_METADATA, - topLevelError)); - } - } else - this.struct.set(ERROR_CODE_KEY_NAME, this.error.code()); - - this.struct.set(RESPONSES_KEY_NAME, getTopicArray(this.responseData).toArray()); - } - - public OffsetFetchResponse(Map responseData) { - this(Errors.NONE, responseData, CURRENT_VERSION); - } - - /** - * Constructor for version 2 and above when there is a coordinator or group level error - * @param topLevelErrorCode Coordinator or group level error code - */ - public OffsetFetchResponse(Errors topLevelError) { - this(topLevelError, new ArrayList(), CURRENT_VERSION); - } - public OffsetFetchResponse(Struct struct) { super(struct); Errors topLevelError = Errors.NONE; - this.responseData = new HashMap(); + this.responseData = new HashMap<>(); for (Object topicResponseObj : struct.getArray(RESPONSES_KEY_NAME)) { Struct topicResponse = (Struct) topicResponseObj; String topic = topicResponse.getString(TOPIC_KEY_NAME); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java index c05643ded01cb..e0e009007ac60 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceRequest.java @@ -20,12 +20,12 @@ import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.MemoryRecords; -import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.CollectionUtils; 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.List; import java.util.Map; @@ -78,7 +78,13 @@ public String toString() { private final short acks; private final int timeout; - private final Map partitionRecords; + + private final Map partitionSizes; + + // This is set to null by `clearPartitionRecords` to prevent unnecessary memory retention when a produce request is + // put in the purgatory (due to client throttling, it can take a while before the response is sent). + // Care should be taken in methods that use this field. + private volatile Map partitionRecords; private ProduceRequest(short version, short acks, int timeout, Map partitionRecords) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.PRODUCE.id, version)), version); @@ -104,6 +110,14 @@ private ProduceRequest(short version, short acks, int timeout, Map createPartitionSizes(Map partitionRecords) { + Map result = new HashMap<>(partitionRecords.size()); + for (Map.Entry entry : partitionRecords.entrySet()) + result.put(entry.getKey(), entry.getValue().sizeInBytes()); + return result; } public ProduceRequest(Struct struct, short version) { @@ -119,10 +133,23 @@ public ProduceRequest(Struct struct, short version) { partitionRecords.put(new TopicPartition(topic, partition), records); } } + partitionSizes = createPartitionSizes(partitionRecords); acks = struct.getShort(ACKS_KEY_NAME); timeout = struct.getInt(TIMEOUT_KEY_NAME); } + @Override + public String toString() { + // Use the same format as `Struct.toString()` + StringBuilder bld = new StringBuilder(); + bld.append("{acks=").append(acks) + .append(",timeout=").append(timeout) + .append(",partitionSizes=") + .append(Utils.mkString(partitionSizes, "[", "]", "=", ",")) + .append("}"); + return bld.toString(); + } + @Override public AbstractResponse getErrorResponse(Throwable e) { /* In case the producer doesn't actually want any response */ @@ -130,10 +157,10 @@ public AbstractResponse getErrorResponse(Throwable e) { return null; Map responseMap = new HashMap<>(); + ProduceResponse.PartitionResponse partitionResponse = new ProduceResponse.PartitionResponse(Errors.forException(e)); - for (Map.Entry entry : partitionRecords.entrySet()) { - responseMap.put(entry.getKey(), new ProduceResponse.PartitionResponse(Errors.forException(e).code(), ProduceResponse.INVALID_OFFSET, Record.NO_TIMESTAMP)); - } + for (TopicPartition tp : partitions()) + responseMap.put(tp, partitionResponse); short versionId = version(); switch (versionId) { @@ -148,6 +175,10 @@ public AbstractResponse getErrorResponse(Throwable e) { } } + private Collection partitions() { + return partitionSizes.keySet(); + } + public short acks() { return acks; } @@ -156,12 +187,23 @@ public int timeout() { return timeout; } - public Map partitionRecords() { + /** + * Returns the partition records or throws IllegalStateException if clearPartitionRecords() has been invoked. + */ + public Map partitionRecordsOrFail() { + // Store it in a local variable to protect against concurrent updates + Map partitionRecords = this.partitionRecords; + if (partitionRecords == null) + throw new IllegalStateException("The partition records are no longer available because " + + "clearPartitionRecords() has been invoked."); return partitionRecords; } public void clearPartitionRecords() { - partitionRecords.clear(); + partitionRecords = null; + // It would be better to make this null, but the change is too large for 0.10.2. In trunk, the struct field + // was removed + struct.clear(); } public static ProduceRequest parse(ByteBuffer buffer, int versionId) { diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java index 71e6ab5cfde24..9eaaadf617e15 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ProduceResponse.java @@ -14,9 +14,11 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.utils.CollectionUtils; import java.nio.ByteBuffer; @@ -48,11 +50,20 @@ public class ProduceResponse extends AbstractResponse { /** * Possible error code: * - * TODO + * CORRUPT_MESSAGE (2) + * UNKNOWN_TOPIC_OR_PARTITION (3) + * NOT_LEADER_FOR_PARTITION (6) + * MESSAGE_TOO_LARGE (10) + * INVALID_TOPIC (17) + * RECORD_LIST_TOO_LARGE (18) + * NOT_ENOUGH_REPLICAS (19) + * NOT_ENOUGH_REPLICAS_AFTER_APPEND (20) + * INVALID_REQUIRED_ACKS (21) + * TOPIC_AUTHORIZATION_FAILED (29) */ private static final String BASE_OFFSET_KEY_NAME = "base_offset"; - private static final String TIMESTAMP_KEY_NAME = "timestamp"; + private static final String LOG_APPEND_TIME_KEY_NAME = "log_append_time"; private final Map responses; private final int throttleTime; @@ -105,11 +116,11 @@ public ProduceResponse(Struct struct) { for (Object partResponse : topicRespStruct.getArray(PARTITION_RESPONSES_KEY_NAME)) { Struct partRespStruct = (Struct) partResponse; int partition = partRespStruct.getInt(PARTITION_KEY_NAME); - short errorCode = partRespStruct.getShort(ERROR_CODE_KEY_NAME); + Errors error = Errors.forCode(partRespStruct.getShort(ERROR_CODE_KEY_NAME)); long offset = partRespStruct.getLong(BASE_OFFSET_KEY_NAME); - long timestamp = partRespStruct.getLong(TIMESTAMP_KEY_NAME); + long logAppendTime = partRespStruct.getLong(LOG_APPEND_TIME_KEY_NAME); TopicPartition tp = new TopicPartition(topic, partition); - responses.put(tp, new PartitionResponse(errorCode, offset, timestamp)); + responses.put(tp, new PartitionResponse(error, offset, logAppendTime)); } } this.throttleTime = struct.getInt(THROTTLE_TIME_KEY_NAME); @@ -126,10 +137,10 @@ private void initCommonFields(Map responses) PartitionResponse part = partitionEntry.getValue(); Struct partStruct = topicData.instance(PARTITION_RESPONSES_KEY_NAME) .set(PARTITION_KEY_NAME, partitionEntry.getKey()) - .set(ERROR_CODE_KEY_NAME, part.errorCode) + .set(ERROR_CODE_KEY_NAME, part.error.code()) .set(BASE_OFFSET_KEY_NAME, part.baseOffset); - if (partStruct.hasField(TIMESTAMP_KEY_NAME)) - partStruct.set(TIMESTAMP_KEY_NAME, part.timestamp); + if (partStruct.hasField(LOG_APPEND_TIME_KEY_NAME)) + partStruct.set(LOG_APPEND_TIME_KEY_NAME, part.logAppendTime); partitionArray.add(partStruct); } topicData.set(PARTITION_RESPONSES_KEY_NAME, partitionArray.toArray()); @@ -147,14 +158,18 @@ public int getThrottleTime() { } public static final class PartitionResponse { - public short errorCode; + public Errors error; public long baseOffset; - public long timestamp; + public long logAppendTime; - public PartitionResponse(short errorCode, long baseOffset, long timestamp) { - this.errorCode = errorCode; + public PartitionResponse(Errors error) { + this(error, INVALID_OFFSET, Record.NO_TIMESTAMP); + } + + public PartitionResponse(Errors error, long baseOffset, long logAppendTime) { + this.error = error; this.baseOffset = baseOffset; - this.timestamp = timestamp; + this.logAppendTime = logAppendTime; } @Override @@ -162,11 +177,11 @@ public String toString() { StringBuilder b = new StringBuilder(); b.append('{'); b.append("error: "); - b.append(errorCode); + b.append(error); b.append(",offset: "); b.append(baseOffset); - b.append(",timestamp: "); - b.append(timestamp); + b.append(",logAppendTime: "); + b.append(logAppendTime); b.append('}'); return b.toString(); } diff --git a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java index 57247c85ba13b..faca68574fb4c 100644 --- a/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java @@ -22,8 +22,17 @@ public class KafkaThread extends Thread { private final Logger log = LoggerFactory.getLogger(getClass()); + public KafkaThread(final String name, boolean daemon) { + super(name); + configureThread(name, daemon); + } + public KafkaThread(final String name, Runnable runnable, boolean daemon) { super(runnable, name); + configureThread(name, daemon); + } + + private void configureThread(final String name, boolean daemon) { setDaemon(daemon); setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { public void uncaughtException(Thread t, Throwable e) { diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index ac8d0786d7bc4..afa85bd23e9b2 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.common.utils; +import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.Closeable; @@ -803,4 +804,55 @@ public static long computeChecksum(ByteBuffer buffer, int start, int size) { return Crc32.crc32(buffer.array(), buffer.arrayOffset() + start, size); } + /** + * Read data from the channel to the given byte buffer until there are no bytes remaining in the buffer. If the end + * of the file is reached while there are bytes remaining in the buffer, an EOFException is thrown. + * + * @param channel File channel containing the data to read from + * @param destinationBuffer The buffer into which bytes are to be transferred + * @param position The file position at which the transfer is to begin; it must be non-negative + * @param description A description of what is being read, this will be included in the EOFException if it is thrown + * + * @throws IllegalArgumentException If position is negative + * @throws EOFException If the end of the file is reached while there are remaining bytes in the destination buffer + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions + */ + public static void readFullyOrFail(FileChannel channel, ByteBuffer destinationBuffer, long position, + String description) throws IOException { + if (position < 0) { + throw new IllegalArgumentException("The file channel position cannot be negative, but it is " + position); + } + int expectedReadBytes = destinationBuffer.remaining(); + readFully(channel, destinationBuffer, position); + if (destinationBuffer.hasRemaining()) { + throw new EOFException(String.format("Failed to read `%s` from file channel `%s`. Expected to read %d bytes, " + + "but reached end of file after reading %d bytes. Started read from position %d.", + description, channel, expectedReadBytes, expectedReadBytes - destinationBuffer.remaining(), position)); + } + } + + /** + * Read data from the channel to the given byte buffer until there are no bytes remaining in the buffer or the end + * of the file has been reached. + * + * @param channel File channel containing the data to read from + * @param destinationBuffer The buffer into which bytes are to be transferred + * @param position The file position at which the transfer is to begin; it must be non-negative + * + * @throws IllegalArgumentException If position is negative + * @throws IOException If an I/O error occurs, see {@link FileChannel#read(ByteBuffer, long)} for details on the + * possible exceptions + */ + public static void readFully(FileChannel channel, ByteBuffer destinationBuffer, long position) throws IOException { + if (position < 0) { + throw new IllegalArgumentException("The file channel position cannot be negative, but it is " + position); + } + long currentPosition = position; + int bytesRead; + do { + bytesRead = channel.read(destinationBuffer, currentPosition); + currentPosition += bytesRead; + } while (bytesRead != -1 && destinationBuffer.hasRemaining()); + } } diff --git a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java index 94f1e763a222c..22a7c1d37ccb4 100644 --- a/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java +++ b/clients/src/main/java/org/apache/kafka/server/policy/CreateTopicPolicy.java @@ -13,42 +13,90 @@ package org.apache.kafka.server.policy; +import org.apache.kafka.common.Configurable; import org.apache.kafka.common.errors.PolicyViolationException; import java.util.Collections; import java.util.List; import java.util.Map; -public interface CreateTopicPolicy { +/** + * An interface for enforcing a policy on create topics requests. + * + * Common use cases are requiring that the replication factor, min.insync.replicas and/or retention settings for a + * topic are within an allowable range. + * + * If create.topic.policy.class.name is defined, Kafka will create an instance of the specified class + * using the default constructor and will then pass the broker configs to its configure() method. During + * broker shutdown, the close() method will be invoked so that resources can be released (if necessary). + */ +public interface CreateTopicPolicy extends Configurable, AutoCloseable { + /** + * Class containing the create request parameters. + */ class RequestMetadata { private final String topic; - private final int numPartitions; - private final short replicationFactor; + private final Integer numPartitions; + private final Short replicationFactor; private final Map> replicasAssignments; private final Map configs; - public RequestMetadata(String topic, int numPartitions, short replicationFactor, - Map> replicasAssignments, Map configs) { + /** + * Create an instance of this class with the provided parameters. + * + * This constructor is public to make testing of CreateTopicPolicy implementations easier. + * + * @param topic the name of the topic to created. + * @param numPartitions the number of partitions to create or null if replicasAssignments is set. + * @param replicationFactor the replication factor for the topic or null if replicaAssignments is set. + * @param replicasAssignments replica assignments or null if numPartitions and replicationFactor is set. The + * assignment is a map from partition id to replica (broker) ids. + * @param configs topic configs for the topic to be created, not including broker defaults. Broker configs are + * passed via the {@code configure()} method of the policy implementation. + */ + public RequestMetadata(String topic, Integer numPartitions, Short replicationFactor, + Map> replicasAssignments, Map configs) { this.topic = topic; this.numPartitions = numPartitions; this.replicationFactor = replicationFactor; - this.replicasAssignments = Collections.unmodifiableMap(replicasAssignments); + this.replicasAssignments = replicasAssignments == null ? null : Collections.unmodifiableMap(replicasAssignments); this.configs = Collections.unmodifiableMap(configs); } + /** + * Return the name of the topic to create. + */ public String topic() { return topic; } - public int numPartitions() { + /** + * Return the number of partitions to create or null if replicaAssignments is not null. + */ + public Integer numPartitions() { return numPartitions; } + /** + * Return the number of replicas to create or null if replicaAssignments is not null. + */ + public Short replicationFactor() { + return replicationFactor; + } + + /** + * Return a map from partition id to replica (broker) ids or null if numPartitions and replicationFactor are + * set instead. + */ public Map> replicasAssignments() { return replicasAssignments; } + /** + * Return topic configs in the request, not including broker defaults. Broker configs are passed via + * the {@code configure()} method of the policy implementation. + */ public Map configs() { return configs; } @@ -63,5 +111,15 @@ public String toString() { } } + /** + * Validate the request parameters and throw a PolicyViolationException with a suitable error + * message if the create request parameters for the provided topic do not satisfy this policy. + * + * Clients will receive the POLICY_VIOLATION error code along with the exception's message. Note that validation + * failure only affects the relevant topic, other topics in the request will still be processed. + * + * @param requestMetadata the create request parameters for the provided topic. + * @throws PolicyViolationException if the request parameters do not satisfy this policy. + */ void validate(RequestMetadata requestMetadata) throws PolicyViolationException; } diff --git a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java index cfd2a941eb9f5..db88b77d3b5b5 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/MetadataTest.java @@ -52,7 +52,7 @@ public void tearDown() { @Test public void testMetadata() throws Exception { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); metadata.requestUpdate(); assertFalse("Still no updated needed due to backoff", metadata.timeToNextUpdate(time) == 0); @@ -67,7 +67,7 @@ public void testMetadata() throws Exception { // This simulates the metadata update sequence in KafkaProducer while (t1.isAlive() || t2.isAlive()) { if (metadata.timeToNextUpdate(time) == 0) { - metadata.update(TestUtils.singletonCluster(topic, 1), time); + metadata.update(TestUtils.singletonCluster(topic, 1), Collections.emptySet(), time); time += refreshBackoffMs; } Thread.sleep(1); @@ -97,7 +97,7 @@ private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataEx assertEquals(0, metadata.timeToNextUpdate(now)); // lastSuccessfulRefreshMs updated to now. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); // The last update was successful so the remaining time to expire the current metadata should be returned. assertEquals(largerOfBackoffAndExpire, metadata.timeToNextUpdate(now)); @@ -108,7 +108,7 @@ private static void checkTimeToNextUpdate(long refreshBackoffMs, long metadataEx assertEquals(refreshBackoffMs, metadata.timeToNextUpdate(now)); // Reset needUpdate to false. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); assertEquals(largerOfBackoffAndExpire, metadata.timeToNextUpdate(now)); // Both metadataExpireMs and refreshBackoffMs elapsed. @@ -152,13 +152,13 @@ public void testTimeToNextUpdate_OverwriteBackoff() { long now = 10000; // New topic added to fetch set and update requested. It should allow immediate update. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.add("new-topic"); assertEquals(0, metadata.timeToNextUpdate(now)); // Even though setTopics called, immediate update isn't necessary if the new topic set isn't // containing a new topic, - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.setTopics(metadata.topics()); assertEquals(metadataExpireMs, metadata.timeToNextUpdate(now)); @@ -167,12 +167,12 @@ public void testTimeToNextUpdate_OverwriteBackoff() { assertEquals(0, metadata.timeToNextUpdate(now)); // If metadata requested for all topics it should allow immediate update. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.needMetadataForAllTopics(true); assertEquals(0, metadata.timeToNextUpdate(now)); // However if metadata is already capable to serve all topics it shouldn't override backoff. - metadata.update(Cluster.empty(), now); + metadata.update(Cluster.empty(), Collections.emptySet(), now); metadata.needMetadataForAllTopics(true); assertEquals(metadataExpireMs, metadata.timeToNextUpdate(now)); } @@ -187,7 +187,7 @@ public void testTimeToNextUpdate_OverwriteBackoff() { @Test public void testMetadataUpdateWaitTime() throws Exception { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("No update needed.", metadata.timeToNextUpdate(time) == 0); // first try with a max wait time of 0 and ensure that this returns back without waiting forever try { @@ -209,7 +209,7 @@ public void testMetadataUpdateWaitTime() throws Exception { @Test public void testFailedUpdate() { long time = 100; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertEquals(100, metadata.timeToNextUpdate(1000)); metadata.failedUpdate(1100); @@ -218,14 +218,14 @@ public void testFailedUpdate() { assertEquals(100, metadata.lastSuccessfulUpdate()); metadata.needMetadataForAllTopics(true); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertEquals(100, metadata.timeToNextUpdate(1000)); } @Test public void testUpdateWithNeedMetadataForAllTopics() { long time = 0; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); metadata.needMetadataForAllTopics(true); final List expectedTopics = Collections.singletonList("topic"); @@ -237,7 +237,7 @@ public void testUpdateWithNeedMetadataForAllTopics() { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertArrayEquals("Metadata got updated with wrong set of topics.", expectedTopics.toArray(), metadata.topics().toArray()); @@ -255,7 +255,7 @@ public void testClusterListenerGetsNotifiedOfUpdate() { String hostName = "www.example.com"; Cluster cluster = Cluster.bootstrap(Arrays.asList(new InetSocketAddress(hostName, 9002))); - metadata.update(cluster, time); + metadata.update(cluster, Collections.emptySet(), time); assertFalse("ClusterResourceListener should not called when metadata is updated with bootstrap Cluster", MockClusterResourceListener.IS_ON_UPDATE_CALLED.get()); @@ -267,7 +267,7 @@ public void testClusterListenerGetsNotifiedOfUpdate() { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("MockClusterResourceListener did not get cluster metadata correctly", "dummy", mockClusterListener.clusterResource().clusterId()); @@ -279,10 +279,10 @@ public void testClusterListenerGetsNotifiedOfUpdate() { public void testListenerGetsNotifiedOfUpdate() { long time = 0; final Set topics = new HashSet<>(); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); metadata.addListener(new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { topics.clear(); topics.addAll(cluster.topics()); } @@ -296,7 +296,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("Listener did not update topics list correctly", new HashSet<>(Arrays.asList("topic", "topic1")), topics); @@ -306,10 +306,10 @@ public void onMetadataUpdate(Cluster cluster) { public void testListenerCanUnregister() { long time = 0; final Set topics = new HashSet<>(); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); final Metadata.Listener listener = new Metadata.Listener() { @Override - public void onMetadataUpdate(Cluster cluster) { + public void onMetadataUpdate(Cluster cluster, Set unavailableTopics) { topics.clear(); topics.addAll(cluster.topics()); } @@ -324,7 +324,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic1", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); metadata.removeListener(listener); @@ -336,7 +336,7 @@ public void onMetadataUpdate(Cluster cluster) { new PartitionInfo("topic3", 0, null, null, null)), Collections.emptySet(), Collections.emptySet()), - 100); + Collections.emptySet(), 100); assertEquals("Listener did not update topics list correctly", new HashSet<>(Arrays.asList("topic", "topic1")), topics); @@ -349,17 +349,17 @@ public void testTopicExpiry() throws Exception { // Test that topic is expired if not used within the expiry interval long time = 0; metadata.add("topic1"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("Unused topic not expired", metadata.containsTopic("topic1")); // Test that topic is not expired if used within the expiry interval metadata.add("topic2"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); for (int i = 0; i < 3; i++) { time += Metadata.TOPIC_EXPIRY_MS / 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Topic expired even though in use", metadata.containsTopic("topic2")); metadata.add("topic2"); } @@ -368,9 +368,9 @@ public void testTopicExpiry() throws Exception { HashSet topics = new HashSet<>(); topics.add("topic4"); metadata.setTopics(topics); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertFalse("Unused topic not expired", metadata.containsTopic("topic4")); } @@ -381,17 +381,17 @@ public void testNonExpiringMetadata() throws Exception { // Test that topic is not expired if not used within the expiry interval long time = 0; metadata.add("topic1"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); time += Metadata.TOPIC_EXPIRY_MS; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Unused topic expired when expiry disabled", metadata.containsTopic("topic1")); // Test that topic is not expired if used within the expiry interval metadata.add("topic2"); - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); for (int i = 0; i < 3; i++) { time += Metadata.TOPIC_EXPIRY_MS / 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Topic expired even though in use", metadata.containsTopic("topic2")); metadata.add("topic2"); } @@ -401,7 +401,7 @@ public void testNonExpiringMetadata() throws Exception { topics.add("topic4"); metadata.setTopics(topics); time += metadataExpireMs * 2; - metadata.update(Cluster.empty(), time); + metadata.update(Cluster.empty(), Collections.emptySet(), time); assertTrue("Unused topic expired when expiry disabled", metadata.containsTopic("topic4")); } diff --git a/clients/src/test/java/org/apache/kafka/clients/MockClient.java b/clients/src/test/java/org/apache/kafka/clients/MockClient.java index bfde9f95b250f..df29f312e8265 100644 --- a/clients/src/test/java/org/apache/kafka/clients/MockClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/MockClient.java @@ -26,6 +26,7 @@ import java.util.ArrayDeque; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; @@ -63,6 +64,7 @@ public FutureResponse(AbstractResponse responseBody, boolean disconnected, Reque private final Time time; private final Metadata metadata; + private Set unavailableTopics; private int correlation = 0; private Node node = null; private final Set ready = new HashSet<>(); @@ -72,16 +74,17 @@ public FutureResponse(AbstractResponse responseBody, boolean disconnected, Reque // Use concurrent queue for responses so that responses may be updated during poll() from a different thread. private final Queue responses = new ConcurrentLinkedDeque<>(); private final Queue futureResponses = new ArrayDeque<>(); - private final Queue metadataUpdates = new ArrayDeque<>(); + private final Queue metadataUpdates = new ArrayDeque<>(); + private volatile NodeApiVersions nodeApiVersions = NodeApiVersions.create(); public MockClient(Time time) { - this.time = time; - this.metadata = null; + this(time, null); } public MockClient(Time time, Metadata metadata) { this.time = time; this.metadata = metadata; + this.unavailableTopics = Collections.emptySet(); } @Override @@ -145,7 +148,8 @@ public void send(ClientRequest request, long now) { FutureResponse futureResp = iterator.next(); if (futureResp.node != null && !request.destination().equals(futureResp.node.idString())) continue; - + request.requestBuilder().setVersion(nodeApiVersions.usableVersion( + request.requestBuilder().apiKey())); AbstractRequest abstractRequest = request.requestBuilder().build(); if (!futureResp.requestMatcher.matches(abstractRequest)) throw new IllegalStateException("Next in line response did not match expected request"); @@ -165,11 +169,13 @@ public List poll(long timeoutMs, long now) { List copy = new ArrayList<>(this.responses); if (metadata != null && metadata.updateRequested()) { - Cluster cluster = metadataUpdates.poll(); - if (cluster == null) - metadata.update(metadata.fetch(), time.milliseconds()); - else - metadata.update(cluster, time.milliseconds()); + MetadataUpdate metadataUpdate = metadataUpdates.poll(); + if (metadataUpdate == null) + metadata.update(metadata.fetch(), this.unavailableTopics, time.milliseconds()); + else { + this.unavailableTopics = metadataUpdate.unavailableTopics; + metadata.update(metadataUpdate.cluster, metadataUpdate.unavailableTopics, time.milliseconds()); + } } while (!this.responses.isEmpty()) { @@ -275,8 +281,8 @@ public void reset() { metadataUpdates.clear(); } - public void prepareMetadataUpdate(Cluster cluster) { - metadataUpdates.add(cluster); + public void prepareMetadataUpdate(Cluster cluster, Set unavailableTopics) { + metadataUpdates.add(new MetadataUpdate(cluster, unavailableTopics)); } public void setNode(Node node) { @@ -334,4 +340,16 @@ public interface RequestMatcher { boolean matches(AbstractRequest body); } + public void setNodeApiVersions(NodeApiVersions nodeApiVersions) { + this.nodeApiVersions = nodeApiVersions; + } + + private static class MetadataUpdate { + final Cluster cluster; + final Set unavailableTopics; + MetadataUpdate(Cluster cluster, Set unavailableTopics) { + this.cluster = cluster; + this.unavailableTopics = unavailableTopics; + } + } } 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 deaf2cc15e14e..a95af83b43e09 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NetworkClientTest.java @@ -78,7 +78,7 @@ private NetworkClient createNetworkClientWithNoVersionDiscovery() { @Before public void setup() { - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); } @Test(expected = IllegalStateException.class) diff --git a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java index b39a0aa88fabd..53c47c833bfb7 100644 --- a/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/NodeApiVersionsTest.java @@ -19,39 +19,50 @@ import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion; import org.junit.Assert; import org.junit.Test; import java.util.ArrayList; import java.util.Collections; +import java.util.LinkedList; import java.util.List; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class NodeApiVersionsTest { @Test public void testUnsupportedVersionsToString() { - NodeApiVersions versions = new NodeApiVersions( - Collections.emptyList()); + NodeApiVersions versions = new NodeApiVersions(Collections.emptyList()); StringBuilder bld = new StringBuilder(); - String prefix = "{"; + String prefix = "("; for (ApiKeys apiKey : ApiKeys.values()) { bld.append(prefix).append(apiKey.name). append("(").append(apiKey.id).append("): UNSUPPORTED"); prefix = ", "; } - bld.append("}"); + bld.append(")"); assertEquals(bld.toString(), versions.toString()); } + @Test + public void testUnknownApiVersionsToString() { + ApiVersion unknownApiVersion = new ApiVersion((short) 337, (short) 0, (short) 1); + NodeApiVersions versions = new NodeApiVersions(Collections.singleton(unknownApiVersion)); + assertTrue(versions.toString().endsWith("UNKNOWN(337): 0 to 1)")); + } + @Test public void testVersionsToString() { List versionList = new ArrayList<>(); for (ApiKeys apiKey : ApiKeys.values()) { if (apiKey == ApiKeys.CONTROLLED_SHUTDOWN_KEY) { versionList.add(new ApiVersion(apiKey.id, (short) 0, (short) 0)); + } else if (apiKey == ApiKeys.DELETE_TOPICS) { + versionList.add(new ApiVersion(apiKey.id, (short) 10000, (short) 10001)); } else { versionList.add(new ApiVersion(apiKey.id, ProtoUtils.oldestVersion(apiKey.id), ProtoUtils.latestVersion(apiKey.id))); @@ -59,11 +70,13 @@ public void testVersionsToString() { } NodeApiVersions versions = new NodeApiVersions(versionList); StringBuilder bld = new StringBuilder(); - String prefix = "{"; + String prefix = "("; for (ApiKeys apiKey : ApiKeys.values()) { bld.append(prefix); if (apiKey == ApiKeys.CONTROLLED_SHUTDOWN_KEY) { - bld.append("ControlledShutdown(7): 0 [usable: NONE]"); + bld.append("ControlledShutdown(7): 0 [unusable: node too old]"); + } else if (apiKey == ApiKeys.DELETE_TOPICS) { + bld.append("DeleteTopics(20): 10000 to 10001 [unusable: node too new]"); } else { bld.append(apiKey.name).append("("). append(apiKey.id).append("): "); @@ -80,7 +93,7 @@ public void testVersionsToString() { } prefix = ", "; } - bld.append("}"); + bld.append(")"); assertEquals(bld.toString(), versions.toString()); } @@ -98,4 +111,25 @@ public void testUsableVersionCalculation() { } assertEquals(2, versions.usableVersion(ApiKeys.FETCH)); } + + @Test(expected = UnsupportedVersionException.class) + public void testUsableVersionCalculationNoKnownVersions() { + List versionList = new ArrayList<>(); + NodeApiVersions versions = new NodeApiVersions(versionList); + versions.usableVersion(ApiKeys.FETCH); + } + + @Test + public void testUsableVersionLatestVersions() { + List versionList = new LinkedList<>(); + for (ApiVersion apiVersion: ApiVersionsResponse.API_VERSIONS_RESPONSE.apiVersions()) { + versionList.add(apiVersion); + } + // Add an API key that we don't know about. + versionList.add(new ApiVersion((short) 100, (short) 0, (short) 1)); + NodeApiVersions versions = new NodeApiVersions(versionList); + for (ApiKeys apiKey: ApiKeys.values()) { + assertEquals(ProtoUtils.latestVersion(apiKey.id), versions.usableVersion(apiKey)); + } + } } 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 2eeed551abdb0..f2905a9c97151 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 @@ -61,6 +61,7 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.MockConsumerInterceptor; import org.apache.kafka.test.MockMetricsReporter; import org.apache.kafka.test.TestUtils; @@ -363,9 +364,9 @@ public void verifyHeartbeatSent() throws Exception { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); - MockClient client = new MockClient(time); + MockClient client = new MockClient(time, metadata); client.setNode(node); PartitionAssignor assignor = new RoundRobinAssignor(); @@ -404,7 +405,7 @@ public void verifyHeartbeatSentWhenFetchedDataReady() throws Exception { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -445,7 +446,7 @@ public void verifyNoCoordinatorLookupForManualAssignmentWithSeek() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -481,7 +482,7 @@ public void testCommitsFetchedDuringAssign() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -529,7 +530,7 @@ public void testAutoCommitSentBeforePositionUpdate() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -590,7 +591,7 @@ public void testRegexSubscription() { consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); consumer.poll(0); assertEquals(singleton(topic), consumer.subscription()); @@ -621,7 +622,7 @@ public void testChangingRegexSubscription() { MockClient client = new MockClient(time, metadata); client.setNode(node); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, false, autoCommitIntervalMs); @@ -629,14 +630,14 @@ public void testChangingRegexSubscription() { Node coordinator = prepareRebalance(client, node, singleton(topic), assignor, singletonList(tp0), null); consumer.subscribe(Pattern.compile(topic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); consumer.poll(0); assertEquals(singleton(topic), consumer.subscription()); consumer.subscribe(Pattern.compile(otherTopic), getConsumerRebalanceListener(consumer)); - client.prepareMetadataUpdate(cluster); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); prepareRebalance(client, node, singleton(otherTopic), assignor, singletonList(otherTopicPartition), coordinator); consumer.poll(0); @@ -659,7 +660,7 @@ public void testWakeupWithFetchDataAvailable() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -704,7 +705,7 @@ public void testPollThrowsInterruptExceptionIfInterrupted() throws Exception { final Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); final MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -744,7 +745,7 @@ public void fetchResponseWithUnexpectedPartitionIsIgnored() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -792,7 +793,7 @@ public void testSubscriptionChangesWithAutoCommitEnabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -912,7 +913,7 @@ public void testSubscriptionChangesWithAutoCommitDisabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -980,7 +981,7 @@ public void testManualAssignmentChangeWithAutoCommitEnabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1045,7 +1046,7 @@ public void testManualAssignmentChangeWithAutoCommitDisabled() { Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); MockClient client = new MockClient(time, metadata); client.setNode(node); @@ -1094,6 +1095,60 @@ public void testManualAssignmentChangeWithAutoCommitDisabled() { consumer.close(); } + @Test + public void testOffsetOfPausedPartitions() { + int rebalanceTimeoutMs = 60000; + int sessionTimeoutMs = 30000; + int heartbeatIntervalMs = 3000; + int autoCommitIntervalMs = 1000; + + Time time = new MockTime(); + Cluster cluster = TestUtils.singletonCluster(topic, 2); + Node node = cluster.nodes().get(0); + + Metadata metadata = new Metadata(0, Long.MAX_VALUE); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + + MockClient client = new MockClient(time, metadata); + client.setNode(node); + PartitionAssignor assignor = new RangeAssignor(); + + final KafkaConsumer consumer = newConsumer(time, client, metadata, assignor, + rebalanceTimeoutMs, sessionTimeoutMs, heartbeatIntervalMs, true, autoCommitIntervalMs); + + // lookup coordinator + client.prepareResponseFrom(new GroupCoordinatorResponse(Errors.NONE.code(), node), node); + Node coordinator = new Node(Integer.MAX_VALUE - node.id(), node.host(), node.port()); + + // manual assignment + Set partitions = Utils.mkSet(tp0, tp1); + consumer.assign(partitions); + // verify consumer's assignment + assertTrue(consumer.assignment().equals(partitions)); + + consumer.pause(partitions); + consumer.seekToEnd(partitions); + + // fetch and verify committed offset of two partitions + Map offsets = new HashMap<>(); + offsets.put(tp0, 0L); + offsets.put(tp1, 0L); + + client.prepareResponseFrom(offsetResponse(offsets, Errors.NONE), coordinator); + assertEquals(0, consumer.committed(tp0).offset()); + assertEquals(0, consumer.committed(tp1).offset()); + + // fetch and verify consumer's position in the two partitions + client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp0, 3L), Errors.NONE.code())); + client.prepareResponse(listOffsetsResponse(Collections.singletonMap(tp1, 3L), Errors.NONE.code())); + assertEquals(3L, consumer.position(tp0)); + assertEquals(3L, consumer.position(tp1)); + + client.requests().clear(); + consumer.unsubscribe(); + consumer.close(); + } + @Test(expected = IllegalStateException.class) public void testPollWithNoSubscription() { KafkaConsumer consumer = newConsumer(); @@ -1171,9 +1226,9 @@ private void consumerCloseTest(final long closeTimeoutMs, Node node = cluster.nodes().get(0); Metadata metadata = new Metadata(0, Long.MAX_VALUE); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); - MockClient client = new MockClient(time); + MockClient client = new MockClient(time, metadata); client.setNode(node); PartitionAssignor assignor = new RoundRobinAssignor(); @@ -1183,6 +1238,8 @@ private void consumerCloseTest(final long closeTimeoutMs, consumer.subscribe(Arrays.asList(topic), getConsumerRebalanceListener(consumer)); Node coordinator = prepareRebalance(client, node, assignor, Arrays.asList(tp0), null); + client.prepareMetadataUpdate(cluster, Collections.emptySet()); + // Poll with responses client.prepareResponseFrom(fetchResponse(tp0, 0, 1), node); client.prepareResponseFrom(fetchResponse(tp0, 1, 0), node); @@ -1370,7 +1427,7 @@ private OffsetFetchResponse offsetResponse(Map offsets, Er for (Map.Entry entry : offsets.entrySet()) { partitionData.put(entry.getKey(), new OffsetFetchResponse.PartitionData(entry.getValue(), "", error)); } - return new OffsetFetchResponse(partitionData); + return new OffsetFetchResponse(Errors.NONE, partitionData); } private ListOffsetResponse listOffsetsResponse(Map offsets, short error) { @@ -1429,12 +1486,11 @@ private KafkaConsumer newConsumer(Time time, Deserializer valueDeserializer = new StringDeserializer(); OffsetResetStrategy autoResetStrategy = OffsetResetStrategy.EARLIEST; - OffsetCommitCallback defaultCommitCallback = new ConsumerCoordinator.DefaultOffsetCommitCallback(); List assignors = Arrays.asList(assignor); ConsumerInterceptors interceptors = null; Metrics metrics = new Metrics(); - SubscriptionState subscriptions = new SubscriptionState(autoResetStrategy, metrics); + SubscriptionState subscriptions = new SubscriptionState(autoResetStrategy); ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, retryBackoffMs, requestTimeoutMs); ConsumerCoordinator consumerCoordinator = new ConsumerCoordinator( consumerClient, @@ -1449,7 +1505,6 @@ private KafkaConsumer newConsumer(Time time, metricGroupPrefix, time, retryBackoffMs, - defaultCommitCallback, autoCommitEnabled, autoCommitIntervalMs, interceptors, 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 bb617ae4f2479..8846b5e5b45da 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 @@ -79,7 +79,7 @@ public void setupCoordinator() { Metrics metrics = new Metrics(); Cluster cluster = TestUtils.singletonCluster("topic", 1); - metadata.update(cluster, mockTime.milliseconds()); + metadata.update(cluster, Collections.emptySet(), mockTime.milliseconds()); this.node = cluster.nodes().get(0); mockClient.setNode(node); 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 9a8c0b93de6d9..66fe76d8a1358 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 @@ -85,9 +85,11 @@ public class ConsumerCoordinatorTest { - private String topicName = "test"; + private String topic1 = "test1"; + private String topic2 = "test2"; private String groupId = "test-group"; - private TopicPartition tp = new TopicPartition(topicName, 0); + private TopicPartition t1p = new TopicPartition(topic1, 0); + private TopicPartition t2p = new TopicPartition(topic2, 0); private int rebalanceTimeoutMs = 60000; private int sessionTimeoutMs = 10000; private int heartbeatIntervalMs = 5000; @@ -98,28 +100,32 @@ public class ConsumerCoordinatorTest { private List assignors = Collections.singletonList(partitionAssignor); private MockTime time; private MockClient client; - private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Cluster cluster = TestUtils.clusterWith(1, new HashMap() { + { + put(topic1, 1); + put(topic2, 1); + } + }); private Node node = cluster.nodes().get(0); private SubscriptionState subscriptions; private Metadata metadata; private Metrics metrics; private ConsumerNetworkClient consumerClient; private MockRebalanceListener rebalanceListener; - private MockCommitCallback defaultOffsetCommitCallback; + private MockCommitCallback mockOffsetCommitCallback; private ConsumerCoordinator coordinator; - @Before public void setup() { this.time = new MockTime(); - this.client = new MockClient(time); - this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST, metrics); + this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); this.metadata = new Metadata(0, Long.MAX_VALUE); - this.metadata.update(cluster, time.milliseconds()); + this.metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + this.client = new MockClient(time, metadata); this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); - this.defaultOffsetCommitCallback = new MockCommitCallback(); + this.mockOffsetCommitCallback = new MockCommitCallback(); this.partitionAssignor.clear(); client.setNode(node); @@ -157,7 +163,7 @@ public void testGroupDescribeUnauthorized() { @Test(expected = GroupAuthorizationException.class) public void testGroupReadUnauthorized() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -215,8 +221,8 @@ public void testIllegalGeneration() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(singleton(topicName), rebalanceListener); - subscriptions.assignFromSubscribed(Collections.singletonList(tp)); + subscriptions.subscribe(singleton(topic1), rebalanceListener); + subscriptions.assignFromSubscribed(Collections.singletonList(t1p)); time.sleep(sessionTimeoutMs); RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat @@ -239,8 +245,8 @@ public void testUnknownConsumerId() { coordinator.ensureCoordinatorReady(); // illegal_generation will cause re-partition - subscriptions.subscribe(singleton(topicName), rebalanceListener); - subscriptions.assignFromSubscribed(Collections.singletonList(tp)); + subscriptions.subscribe(singleton(topic1), rebalanceListener); + subscriptions.assignFromSubscribed(Collections.singletonList(t1p)); time.sleep(sessionTimeoutMs); RequestFuture future = coordinator.sendHeartbeatRequest(); // should send out the heartbeat @@ -282,11 +288,11 @@ public void testCoordinatorDisconnect() { public void testJoinGroupInvalidGroupId() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); - metadata.update(cluster, time.milliseconds()); + metadata.setTopics(singletonList(topic1)); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -300,18 +306,57 @@ public void testJoinGroupInvalidGroupId() { public void testNormalJoinGroupLeader() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); - metadata.update(cluster, time.milliseconds()); + metadata.setTopics(singletonList(topic1)); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + // normal join group + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().containsKey(consumerId); + } + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); + coordinator.poll(time.milliseconds()); + + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); + assertEquals(singleton(topic1), subscriptions.groupSubscription()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(Collections.emptySet(), rebalanceListener.revoked); + assertEquals(1, rebalanceListener.assignedCount); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + + @Test + public void testPatternJoinGroupLeader() { + final String consumerId = "leader"; + + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + + // partially update the metadata with one topic first, + // let the leader to refresh metadata during assignment + metadata.setTopics(singletonList(topic1)); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // normal join group - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(t1p, t2p))); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(new MockClient.RequestMatcher() { @@ -322,37 +367,40 @@ public boolean matches(AbstractRequest body) { sync.generationId() == 1 && sync.groupAssignment().containsKey(consumerId); } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE.code())); + // expect client to force updating the metadata, if yes gives it both topics + client.prepareMetadataUpdate(cluster, Collections.emptySet()); + coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.groupSubscription().size()); + assertEquals(2, subscriptions.subscription().size()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(2, rebalanceListener.assigned.size()); } @Test public void testMetadataRefreshDuringRebalance() { final String consumerId = "leader"; - final String otherTopicName = "otherTopic"; - TopicPartition otherPartition = new TopicPartition(otherTopicName, 0); subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); metadata.needMetadataForAllTopics(true); - metadata.update(cluster, time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); - assertEquals(singleton(topicName), subscriptions.subscription()); + assertEquals(singleton(topic1), subscriptions.subscription()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> initialSubscription = singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(singletonMap(consumerId, singletonList(tp))); + Map> initialSubscription = singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(singletonMap(consumerId, singletonList(t1p))); // the metadata will be updated in flight with a new topic added - final List updatedSubscription = Arrays.asList(topicName, otherTopicName); + final List updatedSubscription = Arrays.asList(topic1, topic2); final Set updatedSubscriptionSet = new HashSet<>(updatedSubscription); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, initialSubscription, Errors.NONE.code())); @@ -362,15 +410,15 @@ public boolean matches(AbstractRequest body) { final Map updatedPartitions = new HashMap<>(); for (String topic : updatedSubscription) updatedPartitions.put(topic, 1); - metadata.update(TestUtils.clusterWith(1, updatedPartitions), time.milliseconds()); + metadata.update(TestUtils.clusterWith(1, updatedPartitions), Collections.emptySet(), time.milliseconds()); return true; } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); - List newAssignment = Arrays.asList(tp, otherPartition); + List newAssignment = Arrays.asList(t1p, t2p); Set newAssignmentSet = new HashSet<>(newAssignment); - Map> updatedSubscriptions = singletonMap(consumerId, Arrays.asList(topicName, otherTopicName)); + Map> updatedSubscriptions = singletonMap(consumerId, Arrays.asList(topic1, topic2)); partitionAssignor.prepare(singletonMap(consumerId, newAssignment)); // we expect to see a second rebalance with the new-found topics @@ -392,7 +440,7 @@ public boolean matches(AbstractRequest body) { assertEquals(updatedSubscriptionSet, subscriptions.subscription()); assertEquals(newAssignmentSet, subscriptions.assignedPartitions()); assertEquals(2, rebalanceListener.revokedCount); - assertEquals(singleton(tp), rebalanceListener.revoked); + assertEquals(singleton(t1p), rebalanceListener.revoked); assertEquals(2, rebalanceListener.assignedCount); assertEquals(newAssignmentSet, rebalanceListener.assigned); } @@ -401,17 +449,17 @@ public boolean matches(AbstractRequest body) { public void testWakeupDuringJoin() { final String consumerId = "leader"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); - metadata.update(cluster, time.milliseconds()); + metadata.setTopics(singletonList(topic1)); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); // prepare only the first half of the join and then trigger the wakeup client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); @@ -424,22 +472,59 @@ public void testWakeupDuringJoin() { } // now complete the second half - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(Collections.emptySet(), rebalanceListener.revoked); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test public void testNormalJoinGroupFollower() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + // normal join group + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(AbstractRequest body) { + SyncGroupRequest sync = (SyncGroupRequest) body; + return sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().isEmpty(); + } + }, syncGroupResponse(singletonList(t1p), Errors.NONE.code())); + + coordinator.joinGroupIfNeeded(); + + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); + assertEquals(singleton(topic1), subscriptions.groupSubscription()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(Collections.emptySet(), rebalanceListener.revoked); + assertEquals(1, rebalanceListener.assignedCount); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + + @Test + public void testPatternJoinGroupFollower() { + final String consumerId = "consumer"; + + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + + // partially update the metadata with one topic first, + // let the leader to refresh metadata during assignment + metadata.setTopics(singletonList(topic1)); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -454,28 +539,31 @@ public boolean matches(AbstractRequest body) { sync.generationId() == 1 && sync.groupAssignment().isEmpty(); } - }, syncGroupResponse(singletonList(tp), Errors.NONE.code())); + }, syncGroupResponse(Arrays.asList(t1p, t2p), Errors.NONE.code())); + // expect client to force updating the metadata, if yes gives it both topics + client.prepareMetadataUpdate(cluster, Collections.emptySet()); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(2, subscriptions.assignedPartitions().size()); + assertEquals(2, subscriptions.subscription().size()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(2, rebalanceListener.assigned.size()); } @Test public void testLeaveGroupOnClose() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); final AtomicBoolean received = new AtomicBoolean(false); @@ -496,13 +584,13 @@ public boolean matches(AbstractRequest body) { public void testMaybeLeaveGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); final AtomicBoolean received = new AtomicBoolean(false); @@ -526,7 +614,7 @@ public boolean matches(AbstractRequest body) { public void testUnexpectedErrorOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -541,7 +629,7 @@ public void testUnexpectedErrorOnSyncGroup() { public void testUnknownMemberIdOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -558,19 +646,19 @@ public boolean matches(AbstractRequest body) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test public void testRebalanceInProgressOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -581,19 +669,19 @@ public void testRebalanceInProgressOnSyncGroup() { // then let the full join/sync finish successfully client.prepareResponse(joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test public void testIllegalGenerationOnSyncGroup() { final String consumerId = "consumer"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -610,12 +698,12 @@ public boolean matches(AbstractRequest body) { return joinRequest.memberId().equals(JoinGroupRequest.UNKNOWN_MEMBER_ID); } }, joinGroupFollowerResponse(2, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); } @Test @@ -623,27 +711,27 @@ public void testMetadataChangeTriggersRebalance() { final String consumerId = "consumer"; // ensure metadata is up-to-date for leader - metadata.setTopics(singletonList(topicName)); - metadata.update(cluster, time.milliseconds()); + metadata.setTopics(singletonList(topic1)); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topicName)); - partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(tp))); + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, singletonList(t1p))); // the leader is responsible for picking up metadata changes and forcing a group rebalance client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.poll(time.milliseconds()); assertFalse(coordinator.needRejoin()); // a new partition is added to the topic - metadata.update(TestUtils.singletonCluster(topicName, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 2), Collections.emptySet(), time.milliseconds()); // we should detect the change and ask for reassignment assertTrue(coordinator.needRejoin()); @@ -663,14 +751,14 @@ public void testUpdateMetadataDuringRebalance() { metadata.setTopics(topics); // we only have metadata for one topic initially - metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topic1, 1), Collections.emptySet(), time.milliseconds()); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // prepare initial rebalance Map> memberSubscriptions = Collections.singletonMap(consumerId, topics); - partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp1))); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Collections.singletonList(tp1))); client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); client.prepareResponse(new MockClient.RequestMatcher() { @@ -684,12 +772,12 @@ public boolean matches(AbstractRequest body) { Map topicPartitionCounts = new HashMap<>(); topicPartitionCounts.put(topic1, 1); topicPartitionCounts.put(topic2, 1); - metadata.update(TestUtils.singletonCluster(topicPartitionCounts), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(topicPartitionCounts), Collections.emptySet(), time.milliseconds()); return true; } return false; } - }, syncGroupResponse(Arrays.asList(tp1), Errors.NONE.code())); + }, syncGroupResponse(Collections.singletonList(tp1), Errors.NONE.code())); // the metadata update should trigger a second rebalance client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code())); @@ -701,12 +789,72 @@ public boolean matches(AbstractRequest body) { assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); } + @Test + public void testRebalanceAfterTopicUnavailableWithSubscribe() { + unavailableTopicTest(false, false, Collections.emptySet()); + } + + @Test + public void testRebalanceAfterTopicUnavailableWithPatternSubscribe() { + unavailableTopicTest(true, false, Collections.emptySet()); + } + + @Test + public void testRebalanceAfterNotMatchingTopicUnavailableWithPatternSSubscribe() { + unavailableTopicTest(true, false, Collections.singleton("notmatching")); + } + + @Test + public void testAssignWithTopicUnavailable() { + unavailableTopicTest(true, false, Collections.emptySet()); + } + + private void unavailableTopicTest(boolean patternSubscribe, boolean assign, Set unavailableTopicsInLastMetadata) { + final String consumerId = "consumer"; + + metadata.setTopics(singletonList(topic1)); + client.prepareMetadataUpdate(Cluster.empty(), Collections.singleton("test1")); + + if (assign) + subscriptions.assignFromUser(singleton(t1p)); + else if (patternSubscribe) + subscriptions.subscribe(Pattern.compile("test.*"), rebalanceListener); + else + subscriptions.subscribe(singleton(topic1), rebalanceListener); + + client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorReady(); + + Map> memberSubscriptions = Collections.singletonMap(consumerId, singletonList(topic1)); + partitionAssignor.prepare(Collections.>emptyMap()); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(Collections.emptyList(), Errors.NONE.code())); + coordinator.poll(time.milliseconds()); + if (!assign) { + assertFalse(coordinator.needRejoin()); + assertEquals(Collections.emptySet(), rebalanceListener.assigned); + } + assertTrue("Metadata refresh not requested for unavailable partitions", metadata.updateRequested()); + + client.prepareMetadataUpdate(cluster, unavailableTopicsInLastMetadata); + client.poll(0, time.milliseconds()); + client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); + coordinator.poll(time.milliseconds()); + + assertFalse("Metadata refresh requested unnecessarily", metadata.updateRequested()); + if (!assign) { + assertFalse(coordinator.needRejoin()); + assertEquals(singleton(t1p), rebalanceListener.assigned); + } + } @Test public void testExcludeInternalTopicsConfigOption() { subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); - metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), Collections.emptySet(), time.milliseconds()); assertFalse(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @@ -716,7 +864,7 @@ public void testIncludeInternalTopicsConfigOption() { coordinator = buildCoordinator(new Metrics(), assignors, false, false); subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); - metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); + metadata.update(TestUtils.singletonCluster(TestUtils.GROUP_METADATA_TOPIC_NAME, 2), Collections.emptySet(), time.milliseconds()); assertTrue(subscriptions.subscription().contains(TestUtils.GROUP_METADATA_TOPIC_NAME)); } @@ -725,36 +873,36 @@ public void testIncludeInternalTopicsConfigOption() { public void testRejoinGroup() { String otherTopic = "otherTopic"; - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); // join the group once client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertEquals(1, rebalanceListener.revokedCount); assertTrue(rebalanceListener.revoked.isEmpty()); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); // and join the group again - subscriptions.subscribe(new HashSet<>(Arrays.asList(topicName, otherTopic)), rebalanceListener); + subscriptions.subscribe(new HashSet<>(Arrays.asList(topic1, otherTopic)), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(2, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertEquals(2, rebalanceListener.revokedCount); - assertEquals(singleton(tp), rebalanceListener.revoked); + assertEquals(singleton(t1p), rebalanceListener.revoked); assertEquals(2, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test public void testDisconnectInJoin() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -763,19 +911,19 @@ public void testDisconnectInJoin() { client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code()), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); assertFalse(coordinator.needRejoin()); - assertEquals(singleton(tp), subscriptions.assignedPartitions()); + assertEquals(singleton(t1p), subscriptions.assignedPartitions()); assertEquals(1, rebalanceListener.revokedCount); assertEquals(1, rebalanceListener.assignedCount); - assertEquals(singleton(tp), rebalanceListener.assigned); + assertEquals(singleton(t1p), rebalanceListener.assigned); } @Test(expected = ApiException.class) public void testInvalidSessionTimeout() { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -787,19 +935,19 @@ public void testInvalidSessionTimeout() { @Test public void testCommitOffsetOnly() { - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -809,22 +957,22 @@ public void testAutoCommitDynamicAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -834,7 +982,7 @@ public void testAutoCommitDynamicAssignmentRebalance() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); @@ -844,16 +992,16 @@ public void testAutoCommitDynamicAssignmentRebalance() { consumerClient.poll(0); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -861,17 +1009,17 @@ public void testAutoCommitManualAssignment() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(singleton(tp)); - subscriptions.seek(tp, 100); + subscriptions.assignFromUser(singleton(t1p)); + subscriptions.seek(t1p, 100); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); time.sleep(autoCommitIntervalMs); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -879,15 +1027,15 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, ConsumerConfig.DEFAULT_EXCLUDE_INTERNAL_TOPICS, true); - subscriptions.assignFromUser(singleton(tp)); - subscriptions.seek(tp, 100); + subscriptions.assignFromUser(singleton(t1p)); + subscriptions.seek(t1p, 100); // no commit initially since coordinator is unknown consumerClient.poll(0); time.sleep(autoCommitIntervalMs); consumerClient.poll(0); - assertNull(subscriptions.committed(tp)); + assertNull(subscriptions.committed(t1p)); // now find the coordinator client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); @@ -895,59 +1043,62 @@ public void testAutoCommitManualAssignmentCoordinatorUnknown() { // sleep only for the retry backoff time.sleep(retryBackoffMs); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); coordinator.poll(time.milliseconds()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test public void testCommitOffsetMetadata() { - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "hello")), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); - assertEquals(100L, subscriptions.committed(tp).offset()); - assertEquals("hello", subscriptions.committed(tp).metadata()); + assertEquals(100L, subscriptions.committed(t1p).offset()); + assertEquals("hello", subscriptions.committed(t1p).metadata()); } @Test public void testCommitOffsetAsyncWithDefaultCallback() { - int invokedBeforeTest = defaultOffsetCommitCallback.invoked; + int invokedBeforeTest = mockOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback); coordinator.invokeCompletedOffsetCommitCallbacks(); - assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); - assertNull(defaultOffsetCommitCallback.exception); + assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); + assertNull(mockOffsetCommitCallback.exception); } @Test public void testCommitAfterLeaveGroup() { // enable auto-assignment - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); client.prepareResponse(joinGroupFollowerResponse(1, "consumer", "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); + + client.prepareMetadataUpdate(cluster, Collections.emptySet()); + coordinator.joinGroupIfNeeded(); // now switch to manual assignment client.prepareResponse(new LeaveGroupResponse(Errors.NONE.code())); subscriptions.unsubscribe(); coordinator.maybeLeaveGroup(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); // the client should not reuse generation/memberId from auto-subscribed generation client.prepareResponse(new MockClient.RequestMatcher() { @@ -957,24 +1108,24 @@ public boolean matches(AbstractRequest body) { return commitRequest.memberId().equals(OffsetCommitRequest.DEFAULT_MEMBER_ID) && commitRequest.generationId() == OffsetCommitRequest.DEFAULT_GENERATION_ID; } - }, offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + }, offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); AtomicBoolean success = new AtomicBoolean(false); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), callback(success)); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(success.get()); } @Test public void testCommitOffsetAsyncFailedWithDefaultCallback() { - int invokedBeforeTest = defaultOffsetCommitCallback.invoked; + int invokedBeforeTest = mockOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), mockOffsetCommitCallback); coordinator.invokeCompletedOffsetCommitCallbacks(); - assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); - assertTrue(defaultOffsetCommitCallback.exception instanceof RetriableCommitFailedException); + assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); + assertTrue(mockOffsetCommitCallback.exception instanceof RetriableCommitFailedException); } @Test @@ -984,8 +1135,8 @@ public void testCommitOffsetAsyncCoordinatorNotAvailable() { // async commit with coordinator not available MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1000,8 +1151,8 @@ public void testCommitOffsetAsyncNotCoordinator() { // async commit with not coordinator MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1016,8 +1167,8 @@ public void testCommitOffsetAsyncDisconnected() { // async commit with coordinator disconnected MockCommitCallback cb = new MockCommitCallback(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code())), true); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), cb); coordinator.invokeCompletedOffsetCommitCallbacks(); assertTrue(coordinator.coordinatorUnknown()); @@ -1031,10 +1182,10 @@ public void testCommitOffsetSyncNotCoordinator() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test @@ -1043,10 +1194,10 @@ public void testCommitOffsetSyncCoordinatorNotAvailable() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test @@ -1055,10 +1206,10 @@ public void testCommitOffsetSyncCoordinatorDisconnected() { coordinator.ensureCoordinatorReady(); // sync commit with coordinator disconnected (should connect, get metadata, and then submit the commit request) - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code())), true); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.NONE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test(expected = KafkaException.class) @@ -1066,8 +1217,8 @@ public void testCommitUnknownTopicOrPartition() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = OffsetMetadataTooLarge.class) @@ -1076,8 +1227,8 @@ public void testCommitOffsetMetadataTooLarge() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.OFFSET_METADATA_TOO_LARGE.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.OFFSET_METADATA_TOO_LARGE.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1086,8 +1237,8 @@ public void testCommitOffsetIllegalGeneration() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.ILLEGAL_GENERATION.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.ILLEGAL_GENERATION.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1096,8 +1247,8 @@ public void testCommitOffsetUnknownMemberId() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN_MEMBER_ID.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN_MEMBER_ID.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = CommitFailedException.class) @@ -1106,8 +1257,8 @@ public void testCommitOffsetRebalanceInProgress() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.REBALANCE_IN_PROGRESS.code()))); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.REBALANCE_IN_PROGRESS.code()))); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L, "metadata")), Long.MAX_VALUE); } @Test(expected = KafkaException.class) @@ -1116,24 +1267,24 @@ public void testCommitOffsetSyncCallbackWithNonRetriableException() { coordinator.ensureCoordinatorReady(); // sync commit with invalid partitions should throw if we have no callback - client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.UNKNOWN.code())), false); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), Long.MAX_VALUE); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(t1p, Errors.UNKNOWN.code())), false); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(100L)), Long.MAX_VALUE); } @Test(expected = IllegalArgumentException.class) public void testCommitSyncNegativeOffset() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - coordinator.commitOffsetsSync(Collections.singletonMap(tp, new OffsetAndMetadata(-1L)), Long.MAX_VALUE); + coordinator.commitOffsetsSync(Collections.singletonMap(t1p, new OffsetAndMetadata(-1L)), Long.MAX_VALUE); } @Test public void testCommitAsyncNegativeOffset() { - int invokedBeforeTest = defaultOffsetCommitCallback.invoked; + int invokedBeforeTest = mockOffsetCommitCallback.invoked; client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(-1L)), null); + coordinator.commitOffsetsAsync(Collections.singletonMap(t1p, new OffsetAndMetadata(-1L)), mockOffsetCommitCallback); coordinator.invokeCompletedOffsetCommitCallbacks(); - assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); - assertTrue(defaultOffsetCommitCallback.exception instanceof IllegalArgumentException); + assertEquals(invokedBeforeTest + 1, mockOffsetCommitCallback.invoked); + assertTrue(mockOffsetCommitCallback.exception instanceof IllegalArgumentException); } @Test @@ -1141,12 +1292,12 @@ public void testRefreshOffset() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1154,13 +1305,13 @@ public void testRefreshOffsetLoadInProgress() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.GROUP_LOAD_IN_PROGRESS)); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1168,7 +1319,7 @@ public void testRefreshOffsetsGroupNotAuthorized() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED)); try { @@ -1184,9 +1335,9 @@ public void testRefreshOffsetUnknownTopicOrPartition() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.UNKNOWN_TOPIC_OR_PARTITION, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); } @@ -1195,14 +1346,14 @@ public void testRefreshOffsetNotCoordinatorForConsumer() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); client.prepareResponse(offsetFetchResponse(Errors.NOT_COORDINATOR_FOR_GROUP)); client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", 100L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", 100L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(100L, subscriptions.committed(tp).offset()); + assertEquals(100L, subscriptions.committed(t1p).offset()); } @Test @@ -1210,12 +1361,12 @@ public void testRefreshOffsetWithNoFetchableOffsets() { client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); subscriptions.needRefreshCommits(); - client.prepareResponse(offsetFetchResponse(tp, Errors.NONE, "", -1L)); + client.prepareResponse(offsetFetchResponse(t1p, Errors.NONE, "", -1L)); coordinator.refreshCommittedOffsetsIfNeeded(); assertFalse(subscriptions.refreshCommitsNeeded()); - assertEquals(null, subscriptions.committed(tp)); + assertEquals(null, subscriptions.committed(t1p)); } @Test @@ -1341,14 +1492,14 @@ private ConsumerCoordinator prepareCoordinatorForCloseTest(boolean useGroupManag client.prepareResponse(groupCoordinatorResponse(node, Errors.NONE.code())); coordinator.ensureCoordinatorReady(); if (useGroupManagement) { - subscriptions.subscribe(singleton(topicName), rebalanceListener); + subscriptions.subscribe(singleton(topic1), rebalanceListener); client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); - client.prepareResponse(syncGroupResponse(singletonList(tp), Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(singletonList(t1p), Errors.NONE.code())); coordinator.joinGroupIfNeeded(); } else - subscriptions.assignFromUser(singleton(tp)); + subscriptions.assignFromUser(singleton(t1p)); - subscriptions.seek(tp, 100); + subscriptions.seek(t1p, 100); coordinator.poll(time.milliseconds()); return coordinator; @@ -1439,7 +1590,6 @@ private ConsumerCoordinator buildCoordinator(Metrics metrics, "consumer" + groupId, time, retryBackoffMs, - defaultOffsetCommitCallback, autoCommitEnabled, autoCommitIntervalMs, null, @@ -1482,12 +1632,12 @@ private OffsetCommitResponse offsetCommitResponse(Map res } private OffsetFetchResponse offsetFetchResponse(Errors topLevelError) { - return new OffsetFetchResponse(topLevelError); + return new OffsetFetchResponse(topLevelError, Collections.emptyMap()); } private OffsetFetchResponse offsetFetchResponse(TopicPartition tp, Errors partitionLevelError, String metadata, long offset) { OffsetFetchResponse.PartitionData data = new OffsetFetchResponse.PartitionData(offset, metadata, partitionLevelError); - return new OffsetFetchResponse(Collections.singletonMap(tp, data)); + return new OffsetFetchResponse(Errors.NONE, Collections.singletonMap(tp, data)); } private OffsetCommitCallback callback(final AtomicBoolean success) { 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 cffa59f7a3989..24ba434325d67 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 @@ -18,6 +18,7 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.MockClient; +import org.apache.kafka.clients.NodeApiVersions; import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.OffsetAndMetadata; @@ -31,11 +32,13 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.InvalidTopicException; +import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.SerializationException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; 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; import org.apache.kafka.common.record.ByteBufferOutputStream; import org.apache.kafka.common.record.CompressionType; @@ -44,6 +47,7 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.FetchRequest; import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; @@ -53,6 +57,7 @@ import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; @@ -80,6 +85,7 @@ public class FetcherTest { private String groupId = "test-group"; private final String metricGroup = "consumer" + groupId + "-fetch-manager-metrics"; private TopicPartition tp = new TopicPartition(topicName, 0); + private TopicPartition tp1 = new TopicPartition(topicName, 1); private int minBytes = 1; private int maxBytes = Integer.MAX_VALUE; private int maxWaitMs = 0; @@ -88,11 +94,11 @@ public class FetcherTest { private MockTime time = new MockTime(1); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); private MockClient client = new MockClient(time, metadata); - private Cluster cluster = TestUtils.singletonCluster(topicName, 1); + private Cluster cluster = TestUtils.singletonCluster(topicName, 2); private Node node = cluster.nodes().get(0); private Metrics metrics = new Metrics(time); - private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST, metrics); - private SubscriptionState subscriptionsNoAutoReset = new SubscriptionState(OffsetResetStrategy.NONE, metrics); + private SubscriptionState subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); + private SubscriptionState subscriptionsNoAutoReset = new SubscriptionState(OffsetResetStrategy.NONE); private static final double EPSILON = 0.0001; private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); @@ -104,7 +110,7 @@ public class FetcherTest { @Before public void setup() throws Exception { - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); client.setNode(node); MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME, 1L); @@ -316,6 +322,60 @@ public void testFetchNonContinuousRecords() { assertEquals(30L, consumerRecords.get(2).offset()); } + /** + * Test the case where the client makes a pre-v3 FetchRequest, but the server replies with only a partial + * request. This happens when a single message is larger than the per-partition limit. + */ + @Test + public void testFetchRequestWhenRecordTooLarge() { + try { + client.setNodeApiVersions(NodeApiVersions.create(Collections.singletonList( + new ApiVersionsResponse.ApiVersion(ApiKeys.FETCH.id, (short) 2, (short) 2)))); + makeFetchRequestWithIncompleteRecord(); + try { + fetcher.fetchedRecords(); + fail("RecordTooLargeException should have been raised"); + } catch (RecordTooLargeException e) { + assertTrue(e.getMessage().startsWith("There are some messages at [Partition=Offset]: ")); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp).longValue()); + } + } finally { + client.setNodeApiVersions(NodeApiVersions.create()); + } + } + + /** + * Test the case where the client makes a post KIP-74 FetchRequest, but the server replies with only a + * partial request. For v3 and later FetchRequests, the implementation of KIP-74 changed the behavior + * so that at least one message is always returned. Therefore, this case should not happen, and it indicates + * that an internal error has taken place. + */ + @Test + public void testFetchRequestInternalError() { + makeFetchRequestWithIncompleteRecord(); + try { + fetcher.fetchedRecords(); + fail("RecordTooLargeException should have been raised"); + } catch (KafkaException e) { + assertTrue(e.getMessage().startsWith("Failed to make progress reading messages")); + // the position should not advance since no data has been returned + assertEquals(0, subscriptions.position(tp).longValue()); + } + } + + private void makeFetchRequestWithIncompleteRecord() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.seek(tp, 0); + assertEquals(1, fetcher.sendFetches()); + assertFalse(fetcher.hasCompletedFetches()); + MemoryRecords partialRecord = MemoryRecords.readableRecords( + ByteBuffer.wrap(new byte[]{0, 0, 0, 0, 0, 0, 0, 0})); + client.prepareResponse(fetchResponse(partialRecord, Errors.NONE.code(), 100L, 0)); + consumerClient.poll(0); + assertTrue(fetcher.hasCompletedFetches()); + } + @Test public void testUnauthorizedTopic() { subscriptions.assignFromUser(singleton(tp)); @@ -544,6 +604,68 @@ public void testUpdateFetchPositionDisconnect() { assertEquals(5, subscriptions.position(tp).longValue()); } + @Test + public void testUpdateFetchPositionOfPausedPartitionsRequiringOffsetReset() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.pause(tp); // paused partition does not have a valid position + subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.LATEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, 1L, 10L)); + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithoutACommittedOffset() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.pause(tp); // paused partition does not have a valid position + + client.prepareResponse(listOffsetRequestMatcher(ListOffsetRequest.EARLIEST_TIMESTAMP), + listOffsetResponse(Errors.NONE, 1L, 0L)); + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(0, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithoutAValidPosition() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.pause(tp); // paused partition does not have a valid position + subscriptions.seek(tp, 10); + + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + + @Test + public void testUpdateFetchPositionOfPausedPartitionsWithAValidPosition() { + subscriptions.assignFromUser(singleton(tp)); + subscriptions.committed(tp, new OffsetAndMetadata(0)); + subscriptions.seek(tp, 10); + subscriptions.pause(tp); // paused partition already has a valid position + + fetcher.updateFetchPositions(singleton(tp)); + + assertFalse(subscriptions.isOffsetResetNeeded(tp)); + assertFalse(subscriptions.isFetchable(tp)); // because tp is paused + assertTrue(subscriptions.hasValidPosition(tp)); + assertEquals(10, subscriptions.position(tp).longValue()); + } + @Test public void testGetAllTopics() { // sending response before request, as getTopicMetadata is a blocking call @@ -642,8 +764,11 @@ public void testFetcherMetrics() { subscriptions.assignFromUser(singleton(tp)); subscriptions.seek(tp, 0); + MetricName maxLagMetric = metrics.metricName("records-lag-max", metricGroup, ""); + MetricName partitionLagMetric = metrics.metricName(tp + ".records-lag", metricGroup, ""); + Map allMetrics = metrics.metrics(); - KafkaMetric recordsFetchLagMax = allMetrics.get(metrics.metricName("records-lag-max", metricGroup, "")); + KafkaMetric recordsFetchLagMax = allMetrics.get(maxLagMetric); // recordsFetchLagMax should be initialized to negative infinity assertEquals(Double.NEGATIVE_INFINITY, recordsFetchLagMax.value(), EPSILON); @@ -652,12 +777,19 @@ public void testFetcherMetrics() { fetchRecords(MemoryRecords.EMPTY, Errors.NONE.code(), 100L, 0); assertEquals(100, recordsFetchLagMax.value(), EPSILON); + KafkaMetric partitionLag = allMetrics.get(partitionLagMetric); + assertEquals(100, partitionLag.value(), EPSILON); + // recordsFetchLagMax should be hw - offset of the last message after receiving a non-empty FetchResponse MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME); for (int v = 0; v < 3; v++) builder.appendWithOffset((long) v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes()); fetchRecords(builder.build(), Errors.NONE.code(), 200L, 0); assertEquals(197, recordsFetchLagMax.value(), EPSILON); + + // verify de-registration of partition lag + subscriptions.unsubscribe(); + assertFalse(allMetrics.containsKey(partitionLagMetric)); } private Map>> fetchRecords(MemoryRecords records, short error, long hw, int throttleTime) { @@ -697,28 +829,28 @@ public void testGetOffsetsForTimes() { } private void testGetOffsetsForTimesWithError(Errors errorForTp0, - Errors errorForTp1, + Errors errorFortp, long offsetForTp0, - long offsetForTp1, + long offsetFortp, Long expectedOffsetForTp0, - Long expectedOffsetForTp1) { + Long expectedOffsetFortp) { client.reset(); TopicPartition tp0 = tp; - TopicPartition tp1 = new TopicPartition(topicName, 1); + TopicPartition tp = new TopicPartition(topicName, 1); // Ensure metadata has both partition. Cluster cluster = TestUtils.clusterWith(2, topicName, 2); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); // First try should fail due to metadata error. client.prepareResponseFrom(listOffsetResponse(tp0, errorForTp0, offsetForTp0, offsetForTp0), cluster.leaderFor(tp0)); - client.prepareResponseFrom(listOffsetResponse(tp1, errorForTp1, offsetForTp1, offsetForTp1), cluster.leaderFor(tp1)); + client.prepareResponseFrom(listOffsetResponse(tp, errorFortp, offsetFortp, offsetFortp), cluster.leaderFor(tp)); // Second try should succeed. client.prepareResponseFrom(listOffsetResponse(tp0, Errors.NONE, offsetForTp0, offsetForTp0), cluster.leaderFor(tp0)); - client.prepareResponseFrom(listOffsetResponse(tp1, Errors.NONE, offsetForTp1, offsetForTp1), cluster.leaderFor(tp1)); + client.prepareResponseFrom(listOffsetResponse(tp, Errors.NONE, offsetFortp, offsetFortp), cluster.leaderFor(tp)); Map timestampToSearch = new HashMap<>(); timestampToSearch.put(tp0, 0L); - timestampToSearch.put(tp1, 0L); + timestampToSearch.put(tp, 0L); Map offsetAndTimestampMap = fetcher.getOffsetsByTimes(timestampToSearch, Long.MAX_VALUE); if (expectedOffsetForTp0 == null) @@ -728,12 +860,85 @@ private void testGetOffsetsForTimesWithError(Errors errorForTp0, assertEquals(expectedOffsetForTp0.longValue(), offsetAndTimestampMap.get(tp0).offset()); } - if (expectedOffsetForTp1 == null) - assertNull(offsetAndTimestampMap.get(tp1)); + if (expectedOffsetFortp == null) + assertNull(offsetAndTimestampMap.get(tp)); else { - assertEquals(expectedOffsetForTp1.longValue(), offsetAndTimestampMap.get(tp1).timestamp()); - assertEquals(expectedOffsetForTp1.longValue(), offsetAndTimestampMap.get(tp1).offset()); + assertEquals(expectedOffsetFortp.longValue(), offsetAndTimestampMap.get(tp).timestamp()); + assertEquals(expectedOffsetFortp.longValue(), offsetAndTimestampMap.get(tp).offset()); + } + } + + @Test + public void testFetchPositionAfterException() { + // verify the advancement in the next fetch offset equals the number of fetched records when + // some fetched partitions cause Exception. This ensures that consumer won't lose record upon exception + subscriptionsNoAutoReset.assignFromUser(Utils.mkSet(tp, tp1)); + subscriptionsNoAutoReset.seek(tp, 1); + subscriptionsNoAutoReset.seek(tp1, 1); + + assertEquals(1, fetcherNoAutoReset.sendFetches()); + + Map partitions = new HashMap<>(); + partitions.put(tp, new FetchResponse.PartitionData(Errors.OFFSET_OUT_OF_RANGE.code(), 100, MemoryRecords.EMPTY)); + partitions.put(tp1, new FetchResponse.PartitionData(Errors.NONE.code(), 100, records)); + client.prepareResponse(new FetchResponse(new LinkedHashMap<>(partitions), 0)); + consumerClient.poll(0); + + List> fetchedRecords = new ArrayList<>(); + List exceptions = new ArrayList<>(); + + try { + for (List> records: fetcherNoAutoReset.fetchedRecords().values()) + fetchedRecords.addAll(records); + } catch (OffsetOutOfRangeException e) { + exceptions.add(e); + } + + assertEquals(fetchedRecords.size(), subscriptionsNoAutoReset.position(tp1) - 1); + + try { + for (List> records: fetcherNoAutoReset.fetchedRecords().values()) + fetchedRecords.addAll(records); + } catch (OffsetOutOfRangeException e) { + exceptions.add(e); } + + assertEquals(4, subscriptionsNoAutoReset.position(tp1).longValue()); + assertEquals(3, fetchedRecords.size()); + + // Should have received one OffsetOutOfRangeException for partition tp + assertEquals(1, exceptions.size()); + OffsetOutOfRangeException e = exceptions.get(0); + assertTrue(e.offsetOutOfRangePartitions().containsKey(tp)); + assertEquals(e.offsetOutOfRangePartitions().size(), 1); + } + + @Test + public void testSeekBeforeException() { + Fetcher fetcher = createFetcher(subscriptionsNoAutoReset, new Metrics(time), 2); + + subscriptionsNoAutoReset.assignFromUser(Utils.mkSet(tp)); + subscriptionsNoAutoReset.seek(tp, 1); + assertEquals(1, fetcher.sendFetches()); + Map partitions = new HashMap<>(); + partitions.put(tp, new FetchResponse.PartitionData(Errors.NONE.code(), 100, records)); + client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0)); + consumerClient.poll(0); + + assertEquals(2, fetcher.fetchedRecords().get(tp).size()); + + subscriptionsNoAutoReset.assignFromUser(Utils.mkSet(tp, tp1)); + subscriptionsNoAutoReset.seek(tp1, 1); + assertEquals(1, fetcher.sendFetches()); + partitions = new HashMap<>(); + partitions.put(tp1, new FetchResponse.PartitionData(Errors.OFFSET_OUT_OF_RANGE.code(), 100, MemoryRecords.EMPTY)); + client.prepareResponse(new FetchResponse(new LinkedHashMap<>(partitions), 0)); + consumerClient.poll(0); + assertEquals(1, fetcher.fetchedRecords().get(tp).size()); + + subscriptionsNoAutoReset.seek(tp1, 10); + // Should not throw OffsetOutOfRangeException after the seek + assertEquals(0, fetcher.fetchedRecords().size()); } private MockClient.RequestMatcher listOffsetRequestMatcher(final long timestamp) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java index ef7575444fece..61a55e262f909 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/SubscriptionStateTest.java @@ -20,16 +20,17 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.clients.consumer.OffsetResetStrategy; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.utils.Utils; import org.junit.Test; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Pattern; -import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -37,7 +38,7 @@ public class SubscriptionStateTest { - private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST, new Metrics()); + private final SubscriptionState state = new SubscriptionState(OffsetResetStrategy.EARLIEST); private final String topic = "test"; private final String topic1 = "test1"; private final TopicPartition tp0 = new TopicPartition(topic, 0); @@ -77,7 +78,7 @@ public void partitionAssignmentChangeOnTopicSubscription() { // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.assignFromSubscribed(asList(t1p0)); + state.assignFromSubscribed(singleton(t1p0)); // assigned partitions should immediately change assertEquals(singleton(t1p0), state.assignedPartitions()); @@ -96,27 +97,60 @@ public void partitionAssignmentChangeOnPatternSubscription() { // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); + state.subscribeFromPattern(new HashSet<>(Collections.singletonList(topic))); // assigned partitions should remain unchanged assertTrue(state.assignedPartitions().isEmpty()); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(singleton(tp1)); // assigned partitions should immediately change assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); + + state.assignFromSubscribed(Collections.singletonList(t1p0)); + // assigned partitions should immediately change + assertEquals(singleton(t1p0), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); state.subscribe(Pattern.compile(".*t"), rebalanceListener); // assigned partitions should remain unchanged - assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(t1p0), state.assignedPartitions()); state.subscribeFromPattern(singleton(topic)); // assigned partitions should remain unchanged - assertEquals(singleton(tp1), state.assignedPartitions()); + assertEquals(singleton(t1p0), state.assignedPartitions()); + + state.assignFromSubscribed(Collections.singletonList(tp0)); + // assigned partitions should immediately change + assertEquals(singleton(tp0), state.assignedPartitions()); + assertEquals(singleton(topic), state.subscription()); state.unsubscribe(); // assigned partitions should immediately change assertTrue(state.assignedPartitions().isEmpty()); } + @Test + public void verifyAssignmentListener() { + final AtomicReference> assignmentRef = new AtomicReference<>(); + state.addListener(new SubscriptionState.Listener() { + @Override + public void onAssignment(Set assignment) { + assignmentRef.set(assignment); + } + }); + Set userAssignment = Utils.mkSet(tp0, tp1); + state.assignFromUser(userAssignment); + assertEquals(userAssignment, assignmentRef.get()); + + state.unsubscribe(); + assertEquals(Collections.emptySet(), assignmentRef.get()); + + Set autoAssignment = Utils.mkSet(t1p0); + state.subscribe(singleton(topic1), rebalanceListener); + state.assignFromSubscribed(autoAssignment); + assertEquals(autoAssignment, assignmentRef.get()); + } + @Test public void partitionReset() { state.assignFromUser(singleton(tp0)); @@ -139,11 +173,11 @@ public void topicSubscription() { assertEquals(1, state.subscription().size()); assertTrue(state.assignedPartitions().isEmpty()); assertTrue(state.partitionsAutoAssigned()); - state.assignFromSubscribed(asList(tp0)); + state.assignFromSubscribed(singleton(tp0)); state.seek(tp0, 1); state.committed(tp0, new OffsetAndMetadata(1)); assertAllPositions(tp0, 1L); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(singleton(tp1)); assertTrue(state.isAssigned(tp1)); assertFalse(state.isAssigned(tp0)); assertFalse(state.isFetchable(tp1)); @@ -173,20 +207,28 @@ public void commitOffsetMetadata() { @Test(expected = IllegalStateException.class) public void invalidPositionUpdate() { state.subscribe(singleton(topic), rebalanceListener); - state.assignFromSubscribed(asList(tp0)); + state.assignFromSubscribed(singleton(tp0)); state.position(tp0, 0); } + @Test(expected = IllegalArgumentException.class) + public void cantAssignPartitionForUnsubscribedTopics() { + state.subscribe(singleton(topic), rebalanceListener); + state.assignFromSubscribed(Collections.singletonList(t1p0)); + } + + @Test(expected = IllegalArgumentException.class) + public void cantAssignPartitionForUnmatchedPattern() { + state.subscribe(Pattern.compile(".*t"), rebalanceListener); + state.subscribeFromPattern(new HashSet<>(Collections.singletonList(topic))); + state.assignFromSubscribed(Collections.singletonList(t1p0)); + } + @Test(expected = IllegalStateException.class) public void cantChangePositionForNonAssignedPartition() { state.position(tp0, 1); } - public void assertAllPositions(TopicPartition tp, Long offset) { - assertEquals(offset.longValue(), state.committed(tp).offset()); - assertEquals(offset, state.position(tp)); - } - @Test(expected = IllegalStateException.class) public void cantSubscribeTopicAndPattern() { state.subscribe(singleton(topic), rebalanceListener); @@ -215,9 +257,7 @@ public void cantSubscribePatternAndPartition() { public void patternSubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); - - assertEquals( - "Expected subscribed topics count is incorrect", 2, state.subscription().size()); + assertEquals("Expected subscribed topics count is incorrect", 2, state.subscription().size()); } @Test @@ -240,7 +280,7 @@ public void unsubscribeUserSubscribe() { public void unsubscription() { state.subscribe(Pattern.compile(".*"), rebalanceListener); state.subscribeFromPattern(new HashSet<>(Arrays.asList(topic, topic1))); - state.assignFromSubscribed(asList(tp1)); + state.assignFromSubscribed(singleton(tp1)); assertEquals(singleton(tp1), state.assignedPartitions()); state.unsubscribe(); @@ -255,6 +295,11 @@ public void unsubscription() { assertTrue(state.assignedPartitions().isEmpty()); } + private void assertAllPositions(TopicPartition tp, Long offset) { + assertEquals(offset.longValue(), state.committed(tp).offset()); + assertEquals(offset, state.position(tp)); + } + private static class MockRebalanceListener implements ConsumerRebalanceListener { public Collection revoked; public Collection assigned; diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java index 90256bbdddc55..13f7fdab0dca9 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/KafkaProducerTest.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.StringSerializer; @@ -52,6 +53,20 @@ @PowerMockIgnore("javax.management.*") public class KafkaProducerTest { + @Test + public void testConstructorWithSerializers() { + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + new KafkaProducer<>(producerProps, new ByteArraySerializer(), new ByteArraySerializer()).close(); + } + + @Test(expected = ConfigException.class) + public void testNoSerializerProvided() { + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + new KafkaProducer(producerProps); + } + @Test public void testConstructorFailureCloseResource() { Properties props = new Properties(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java index d820dab5f6fe8..bc8105bff5b6b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java @@ -35,16 +35,16 @@ public class RecordSendTest { - private TopicPartition topicPartition = new TopicPartition("test", 0); - private long baseOffset = 45; - private long relOffset = 5; + private final TopicPartition topicPartition = new TopicPartition("test", 0); + private final long baseOffset = 45; + private final long relOffset = 5; /** * Test that waiting on a request that never completes times out */ @Test public void testTimeout() throws Exception { - ProduceRequestResult request = new ProduceRequestResult(); + ProduceRequestResult request = new ProduceRequestResult(topicPartition); FutureRecordMetadata future = new FutureRecordMetadata(request, relOffset, Record.NO_TIMESTAMP, 0, 0, 0); assertFalse("Request is not completed", future.isDone()); @@ -54,7 +54,8 @@ public void testTimeout() throws Exception { } catch (TimeoutException e) { /* this is good */ } - request.done(topicPartition, baseOffset, null); + request.set(baseOffset, Record.NO_TIMESTAMP, null); + request.done(); assertTrue(future.isDone()); assertEquals(baseOffset + relOffset, future.get().offset()); } @@ -81,15 +82,14 @@ public void testBlocking() throws Exception { /* create a new request result that will be completed after the given timeout */ public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeException error, final long timeout) { - final ProduceRequestResult request = new ProduceRequestResult(); + final ProduceRequestResult request = new ProduceRequestResult(topicPartition); Thread thread = new Thread() { public void run() { try { sleep(timeout); - request.done(topicPartition, baseOffset, error); - } catch (InterruptedException e) { - e.printStackTrace(); - } + request.set(baseOffset, Record.NO_TIMESTAMP, error); + request.done(); + } catch (InterruptedException e) { } } }; thread.start(); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 04e14112d3291..f8bb1e9942d19 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.CompressionType; import org.apache.kafka.common.record.LogEntry; @@ -38,10 +39,12 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -409,6 +412,48 @@ public void testExpiredBatches() throws InterruptedException { assertEquals("The batch should be expired when the partition is not muted.", 1, expiredBatches.size()); } + @Test + public void testAppendInExpiryCallback() throws InterruptedException { + long retryBackoffMs = 100L; + long lingerMs = 3000L; + int requestTimeout = 60; + int messagesPerBatch = 1024 / msgSize; + + final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, + retryBackoffMs, metrics, time); + final AtomicInteger expiryCallbackCount = new AtomicInteger(); + final AtomicReference unexpectedException = new AtomicReference(); + Callback callback = new Callback() { + @Override + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception instanceof TimeoutException) { + expiryCallbackCount.incrementAndGet(); + try { + accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); + } catch (InterruptedException e) { + throw new RuntimeException("Unexpected interruption", e); + } + } else if (exception != null) + unexpectedException.compareAndSet(null, exception); + } + }; + + for (int i = 0; i < messagesPerBatch + 1; i++) + accum.append(tp1, 0L, key, value, callback, maxBlockTimeMs); + + assertEquals(2, accum.batches().get(tp1).size()); + assertTrue("First batch not full", accum.batches().get(tp1).peekFirst().isFull()); + + // Advance the clock to expire the first batch. + time.sleep(requestTimeout + 1); + List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); + assertEquals("The batch was not expired", 1, expiredBatches.size()); + assertEquals("Callbacks not invoked for expiry", messagesPerBatch, expiryCallbackCount.get()); + assertNull("Unexpected exception", unexpectedException.get()); + assertEquals("Some messages not appended from expiry callbacks", 2, accum.batches().get(tp1).size()); + assertTrue("First batch not full after expiry callbacks with appends", accum.batches().get(tp1).peekFirst().isFull()); + } + @Test public void testMutedPartitions() throws InterruptedException { long now = time.milliseconds(); 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 00c536c0c4be6..a3dae6688bda3 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 @@ -84,7 +84,7 @@ public void setup() { time, REQUEST_TIMEOUT); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); } @After @@ -197,7 +197,7 @@ public void testSendInOrder() throws Exception { // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); - metadata.update(cluster1, time.milliseconds()); + metadata.update(cluster1, Collections.emptySet(), time.milliseconds()); // Send the first message. TopicPartition tp2 = new TopicPartition("test", 1); @@ -216,7 +216,7 @@ public void testSendInOrder() throws Exception { // Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0 Cluster cluster2 = TestUtils.singletonCluster("test", 2); - metadata.update(cluster2, time.milliseconds()); + metadata.update(cluster2, Collections.emptySet(), time.milliseconds()); // Sender should not send the second message to node 0. sender.run(time.milliseconds()); assertEquals(1, client.inFlightRequestCount()); @@ -232,12 +232,12 @@ public void testSendInOrder() throws Exception { @Test public void testMetadataTopicExpiry() throws Exception { long offset = 0; - metadata.update(Cluster.empty(), time.milliseconds()); + metadata.update(Cluster.empty(), Collections.emptySet(), time.milliseconds()); Future future = accumulator.append(tp, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); assertTrue("Topic not added to metadata", metadata.containsTopic(tp.topic())); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); sender.run(time.milliseconds()); // send produce request client.respond(produceResponse(tp, offset++, Errors.NONE.code(), 0)); sender.run(time.milliseconds()); @@ -247,12 +247,12 @@ public void testMetadataTopicExpiry() throws Exception { assertTrue("Topic not retained in metadata list", metadata.containsTopic(tp.topic())); time.sleep(Metadata.TOPIC_EXPIRY_MS); - metadata.update(Cluster.empty(), time.milliseconds()); + metadata.update(Cluster.empty(), Collections.emptySet(), time.milliseconds()); assertFalse("Unused topic has not been expired", metadata.containsTopic(tp.topic())); future = accumulator.append(tp, time.milliseconds(), "key".getBytes(), "value".getBytes(), null, MAX_BLOCK_TIMEOUT).future; sender.run(time.milliseconds()); assertTrue("Topic not added to metadata", metadata.containsTopic(tp.topic())); - metadata.update(cluster, time.milliseconds()); + metadata.update(cluster, Collections.emptySet(), time.milliseconds()); sender.run(time.milliseconds()); // send produce request client.respond(produceResponse(tp, offset++, Errors.NONE.code(), 0)); sender.run(time.milliseconds()); @@ -272,7 +272,8 @@ private void completedWithError(Future future, Errors error) thr } private ProduceResponse produceResponse(TopicPartition tp, long offset, int error, int throttleTimeMs) { - ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse((short) error, offset, Record.NO_TIMESTAMP); + ProduceResponse.PartitionResponse resp = new ProduceResponse.PartitionResponse(Errors.forCode((short) error), + offset, Record.NO_TIMESTAMP); Map partResp = Collections.singletonMap(tp, resp); return new ProduceResponse(partResp, throttleTimeMs); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java index dcd3befbd1eea..274bf9d1d224f 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java @@ -381,11 +381,11 @@ private void verifyConvertedMessageSet(List initialEntries, Records co } private static List shallowEntries(Records buffer) { - return TestUtils.toList(buffer.shallowEntries().iterator()); + return TestUtils.toList(buffer.shallowEntries()); } private static List deepEntries(Records buffer) { - return TestUtils.toList(buffer.deepEntries().iterator()); + return TestUtils.toList(buffer.deepEntries()); } private FileRecords createFileRecords(Record ... records) throws IOException { diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java index 034faf631dfd1..02ee75e00bcdd 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java @@ -16,6 +16,7 @@ **/ package org.apache.kafka.common.record; +import org.apache.kafka.test.TestUtils; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -28,6 +29,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; @RunWith(value = Parameterized.class) public class MemoryRecordsBuilderTest { @@ -177,6 +179,33 @@ public void buildUsingCreateTime() { } } + @Test + public void testSmallWriteLimit() { + // with a small write limit, we always allow at least one record to be added + + byte[] key = "foo".getBytes(); + byte[] value = "bar".getBytes(); + int writeLimit = 0; + ByteBuffer buffer = ByteBuffer.allocate(512); + MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.CURRENT_MAGIC_VALUE, compressionType, + TimestampType.CREATE_TIME, 0L, Record.NO_TIMESTAMP, writeLimit); + + assertFalse(builder.isFull()); + assertTrue(builder.hasRoomFor(key, value)); + builder.append(0L, key, value); + + assertTrue(builder.isFull()); + assertFalse(builder.hasRoomFor(key, value)); + + MemoryRecords memRecords = builder.build(); + List records = TestUtils.toList(memRecords.records()); + assertEquals(1, records.size()); + + Record record = records.get(0); + assertEquals(ByteBuffer.wrap(key), record.key()); + assertEquals(ByteBuffer.wrap(value), record.value()); + } + @Test public void writePastLimit() { ByteBuffer buffer = ByteBuffer.allocate(64); diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index 9c8ca7f290e8c..9271a3fa9e120 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -135,7 +135,7 @@ public void testFilterTo() { MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); - List shallowEntries = TestUtils.toList(filteredRecords.shallowEntries().iterator()); + List shallowEntries = TestUtils.toList(filteredRecords.shallowEntries()); List expectedOffsets = compression == CompressionType.NONE ? asList(1L, 4L, 5L, 6L) : asList(1L, 5L, 6L); assertEquals(expectedOffsets.size(), shallowEntries.size()); @@ -148,7 +148,7 @@ public void testFilterTo() { shallowEntry.record().timestampType()); } - List deepEntries = TestUtils.toList(filteredRecords.deepEntries().iterator()); + List deepEntries = TestUtils.toList(filteredRecords.deepEntries()); assertEquals(4, deepEntries.size()); LogEntry first = deepEntries.get(0); @@ -197,7 +197,7 @@ public void testFilterToPreservesLogAppendTime() { filtered.flip(); MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered); - List shallowEntries = TestUtils.toList(filteredRecords.shallowEntries().iterator()); + List shallowEntries = TestUtils.toList(filteredRecords.shallowEntries()); assertEquals(compression == CompressionType.NONE ? 3 : 2, shallowEntries.size()); for (LogEntry shallowEntry : shallowEntries) { 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 0d3a1a8a38053..47313c4c37c12 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 @@ -15,9 +15,10 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.errors.ObsoleteBrokerException; import org.apache.kafka.common.errors.NotCoordinatorForGroupException; +import org.apache.kafka.common.errors.NotEnoughReplicasException; import org.apache.kafka.common.errors.UnknownServerException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.network.ListenerName; import org.apache.kafka.common.network.Send; import org.apache.kafka.common.protocol.ApiKeys; @@ -43,8 +44,10 @@ import java.util.Map; import java.util.Set; +import static java.util.Collections.singletonList; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class RequestResponseTest { @@ -90,8 +93,13 @@ public void testSerialization() throws Exception { checkSerialization(createOffsetCommitResponse(), null); checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); checkSerialization(OffsetFetchRequest.forAllPartitions("group1").getErrorResponse(new NotCoordinatorForGroupException()), 2); - checkSerialization(createOffsetFetchRequest()); - checkSerialization(createOffsetFetchRequest().getErrorResponse(new UnknownServerException()), null); + checkSerialization(createOffsetFetchRequest(0)); + checkSerialization(createOffsetFetchRequest(1)); + checkSerialization(createOffsetFetchRequest(2)); + checkSerialization(OffsetFetchRequest.forAllPartitions("group1")); + checkSerialization(createOffsetFetchRequest(0).getErrorResponse(new UnknownServerException()), 0); + checkSerialization(createOffsetFetchRequest(1).getErrorResponse(new UnknownServerException()), 1); + checkSerialization(createOffsetFetchRequest(2).getErrorResponse(new UnknownServerException()), 2); checkSerialization(createOffsetFetchResponse(), null); checkSerialization(createProduceRequest()); checkSerialization(createProduceRequest().getErrorResponse(new UnknownServerException()), null); @@ -174,10 +182,52 @@ private void checkSerialization(AbstractRequestResponse req, Integer version) th req.hashCode(), deserialized.hashCode()); } + @Test + public void produceRequestToStringTest() { + ProduceRequest request = createProduceRequest(); + assertEquals(1, request.partitionRecordsOrFail().size()); + assertTrue(request.toString().contains("partitionSizes")); + + request.clearPartitionRecords(); + try { + request.partitionRecordsOrFail(); + fail("partitionRecordsOrFail should fail after clearPartitionRecords()"); + } catch (IllegalStateException e) { + // OK + } + + // `toString` should behave the same after `clearPartitionRecords` + assertTrue(request.toString().contains("partitionSizes")); + } + + @Test + public void produceRequestGetErrorResponseTest() { + ProduceRequest request = createProduceRequest(); + Set partitions = new HashSet<>(request.partitionRecordsOrFail().keySet()); + + ProduceResponse errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); + assertEquals(partitions, errorResponse.responses().keySet()); + ProduceResponse.PartitionResponse partitionResponse = errorResponse.responses().values().iterator().next(); + assertEquals(Errors.NOT_ENOUGH_REPLICAS, partitionResponse.error); + assertEquals(ProduceResponse.INVALID_OFFSET, partitionResponse.baseOffset); + assertEquals(Record.NO_TIMESTAMP, partitionResponse.logAppendTime); + + request.clearPartitionRecords(); + + // `getErrorResponse` should behave the same after `clearPartitionRecords` + errorResponse = (ProduceResponse) request.getErrorResponse(new NotEnoughReplicasException()); + assertEquals(partitions, errorResponse.responses().keySet()); + partitionResponse = errorResponse.responses().values().iterator().next(); + assertEquals(Errors.NOT_ENOUGH_REPLICAS, partitionResponse.error); + assertEquals(ProduceResponse.INVALID_OFFSET, partitionResponse.baseOffset); + assertEquals(Record.NO_TIMESTAMP, partitionResponse.logAppendTime); + } + @Test public void produceResponseVersionTest() { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, + 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); ProduceResponse v1Response = new ProduceResponse(responseData, 10, 1); ProduceResponse v2Response = new ProduceResponse(responseData, 10, 2); @@ -262,7 +312,7 @@ public void testRequestHeaderWithNullClientId() { assertEquals("", deserialized.clientId()); // null is defaulted to "" } - @Test(expected = ObsoleteBrokerException.class) + @Test(expected = UnsupportedVersionException.class) public void testCreateTopicRequestV0FailsIfValidateOnly() { createCreateTopicRequest(0, true); } @@ -337,7 +387,7 @@ private ListGroupsResponse createListGroupsResponse() { } private DescribeGroupsRequest createDescribeGroupRequest() { - return new DescribeGroupsRequest.Builder(Collections.singletonList("test-group")).build(); + return new DescribeGroupsRequest.Builder(singletonList("test-group")).build(); } private DescribeGroupsResponse createDescribeGroupResponse() { @@ -428,16 +478,17 @@ private OffsetCommitResponse createOffsetCommitResponse() { return new OffsetCommitResponse(responseData); } - private OffsetFetchRequest createOffsetFetchRequest() { - return new OffsetFetchRequest.Builder("group1", - Arrays.asList(new TopicPartition("test11", 1))).build(); + private OffsetFetchRequest createOffsetFetchRequest(int version) { + return new OffsetFetchRequest.Builder("group1", singletonList(new TopicPartition("test11", 1))) + .setVersion((short) version) + .build(); } private OffsetFetchResponse createOffsetFetchResponse() { Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE)); responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(100L, null, Errors.NONE)); - return new OffsetFetchResponse(responseData); + return new OffsetFetchResponse(Errors.NONE, responseData); } private ProduceRequest createProduceRequest() { @@ -448,7 +499,8 @@ private ProduceRequest createProduceRequest() { private ProduceResponse createProduceResponse() { Map responseData = new HashMap<>(); - responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); + responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE, + 10000, Record.NO_TIMESTAMP)); return new ProduceResponse(responseData, 0); } @@ -544,7 +596,7 @@ private SaslHandshakeRequest createSaslHandshakeRequest() { } private SaslHandshakeResponse createSaslHandshakeResponse() { - return new SaslHandshakeResponse(Errors.NONE.code(), Collections.singletonList("GSSAPI")); + return new SaslHandshakeResponse(Errors.NONE.code(), singletonList("GSSAPI")); } private ApiVersionsRequest createApiVersionRequest() { diff --git a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java index 46400b4a63474..194cad6807422 100755 --- a/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/utils/UtilsTest.java @@ -16,18 +16,27 @@ */ package org.apache.kafka.common.utils; +import java.io.EOFException; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.Collections; import java.io.Closeable; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Random; +import org.apache.kafka.test.TestUtils; +import org.easymock.EasyMock; +import org.easymock.IAnswer; import org.junit.Test; + +import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; -import static org.apache.kafka.common.utils.Utils.formatAddress; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -158,6 +167,135 @@ public void testCloseAll() { } } + @Test + public void testReadFullyOrFailWithRealFile() throws IOException { + try (FileChannel channel = FileChannel.open(TestUtils.tempFile().toPath(), StandardOpenOption.READ, StandardOpenOption.WRITE)) { + // prepare channel + String msg = "hello, world"; + channel.write(ByteBuffer.wrap(msg.getBytes()), 0); + channel.force(true); + assertEquals("Message should be written to the file channel", channel.size(), msg.length()); + + ByteBuffer perfectBuffer = ByteBuffer.allocate(msg.length()); + ByteBuffer smallBuffer = ByteBuffer.allocate(5); + ByteBuffer largeBuffer = ByteBuffer.allocate(msg.length() + 1); + // Scenario 1: test reading into a perfectly-sized buffer + Utils.readFullyOrFail(channel, perfectBuffer, 0, "perfect"); + assertFalse("Buffer should be filled up", perfectBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", msg, new String(perfectBuffer.array())); + // Scenario 2: test reading into a smaller buffer + Utils.readFullyOrFail(channel, smallBuffer, 0, "small"); + assertFalse("Buffer should be filled", smallBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", "hello", new String(smallBuffer.array())); + // Scenario 3: test reading starting from a non-zero position + smallBuffer.clear(); + Utils.readFullyOrFail(channel, smallBuffer, 7, "small"); + assertFalse("Buffer should be filled", smallBuffer.hasRemaining()); + assertEquals("Buffer should be populated correctly", "world", new String(smallBuffer.array())); + // Scenario 4: test end of stream is reached before buffer is filled up + try { + Utils.readFullyOrFail(channel, largeBuffer, 0, "large"); + fail("Expected EOFException to be raised"); + } catch (EOFException e) { + // expected + } + } + } + + /** + * Tests that `readFullyOrFail` behaves correctly if multiple `FileChannel.read` operations are required to fill + * the destination buffer. + */ + @Test + public void testReadFullyOrFailWithPartialFileChannelReads() throws IOException { + FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + StringBuilder expectedBufferContent = new StringBuilder(); + fileChannelMockExpectReadWithRandomBytes(channelMock, expectedBufferContent, bufferSize); + EasyMock.replay(channelMock); + Utils.readFullyOrFail(channelMock, buffer, 0L, "test"); + assertEquals("The buffer should be populated correctly", expectedBufferContent.toString(), + new String(buffer.array())); + assertFalse("The buffer should be filled", buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + /** + * Tests that `readFullyOrFail` behaves correctly if multiple `FileChannel.read` operations are required to fill + * the destination buffer. + */ + @Test + public void testReadFullyWithPartialFileChannelReads() throws IOException { + FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + StringBuilder expectedBufferContent = new StringBuilder(); + fileChannelMockExpectReadWithRandomBytes(channelMock, expectedBufferContent, bufferSize); + EasyMock.replay(channelMock); + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + Utils.readFully(channelMock, buffer, 0L); + assertEquals("The buffer should be populated correctly.", expectedBufferContent.toString(), + new String(buffer.array())); + assertFalse("The buffer should be filled", buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + @Test + public void testReadFullyIfEofIsReached() throws IOException { + final FileChannel channelMock = EasyMock.createMock(FileChannel.class); + final int bufferSize = 100; + final String fileChannelContent = "abcdefghkl"; + ByteBuffer buffer = ByteBuffer.allocate(bufferSize); + EasyMock.expect(channelMock.size()).andReturn((long) fileChannelContent.length()); + EasyMock.expect(channelMock.read(EasyMock.anyObject(ByteBuffer.class), EasyMock.anyInt())).andAnswer(new IAnswer() { + @Override + public Integer answer() throws Throwable { + ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0]; + buffer.put(fileChannelContent.getBytes()); + return -1; + } + }); + EasyMock.replay(channelMock); + Utils.readFully(channelMock, buffer, 0L); + assertEquals("abcdefghkl", new String(buffer.array(), 0, buffer.position())); + assertEquals(buffer.position(), channelMock.size()); + assertTrue(buffer.hasRemaining()); + EasyMock.verify(channelMock); + } + + /** + * Expectation setter for multiple reads where each one reads random bytes to the buffer. + * + * @param channelMock The mocked FileChannel object + * @param expectedBufferContent buffer that will be updated to contain the expected buffer content after each + * `FileChannel.read` invocation + * @param bufferSize The buffer size + * @throws IOException If an I/O error occurs + */ + private void fileChannelMockExpectReadWithRandomBytes(final FileChannel channelMock, + final StringBuilder expectedBufferContent, + final int bufferSize) throws IOException { + final int step = 20; + final Random random = new Random(); + int remainingBytes = bufferSize; + while (remainingBytes > 0) { + final int mockedBytesRead = remainingBytes < step ? remainingBytes : random.nextInt(step); + final StringBuilder sb = new StringBuilder(); + EasyMock.expect(channelMock.read(EasyMock.anyObject(ByteBuffer.class), EasyMock.anyInt())).andAnswer(new IAnswer() { + @Override + public Integer answer() throws Throwable { + ByteBuffer buffer = (ByteBuffer) EasyMock.getCurrentArguments()[0]; + for (int i = 0; i < mockedBytesRead; i++) + sb.append("a"); + buffer.put(sb.toString().getBytes()); + expectedBufferContent.append(sb); + return mockedBytesRead; + } + }); + remainingBytes -= mockedBytesRead; + } + } + private static class TestCloseable implements Closeable { private final int id; private final IOException closeException; 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 c39f402bb0ede..0cb32be916b2e 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -31,7 +31,6 @@ import java.nio.file.Path; import java.util.ArrayList; import java.util.Collections; -import java.util.Iterator; import java.util.List; import java.util.Map; import java.util.Properties; @@ -43,7 +42,6 @@ import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -290,28 +288,18 @@ public static void isValidClusterId(String clusterId) { } } - /** - * Throw an exception if the two iterators are of differing lengths or contain - * different messages on their Nth element - */ - public static void checkEquals(Iterator s1, Iterator s2) { - while (s1.hasNext() && s2.hasNext()) - assertEquals(s1.next(), s2.next()); - assertFalse("Iterators have uneven length--first has more", s1.hasNext()); - assertFalse("Iterators have uneven length--second has more", s2.hasNext()); - } - /** * Checks the two iterables for equality by first converting both to a list. */ public static void checkEquals(Iterable it1, Iterable it2) { - assertEquals(toList(it1.iterator()), toList(it2.iterator())); + assertEquals(toList(it1), toList(it2)); } - public static List toList(Iterator iterator) { - List res = new ArrayList<>(); - while (iterator.hasNext()) - res.add(iterator.next()); - return res; + public static List toList(Iterable iterable) { + List list = new ArrayList<>(); + for (T item : iterable) + list.add(item); + return list; } + } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java index 6236f7e93682d..f3487c07017de 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectRecord.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.connector; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.data.Schema; /** @@ -28,7 +27,6 @@ * notion of offset, it is not included here because they differ in type. *

    */ -@InterfaceStability.Unstable public abstract class ConnectRecord> { private final String topic; private final Integer kafkaPartition; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java index 137015603c59b..a46babd352660 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.connector; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigValue; @@ -42,7 +41,6 @@ * Tasks. *

    */ -@InterfaceStability.Unstable public abstract class Connector { protected ConnectorContext context; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java index c8a06e8e1a238..3b887c19e9704 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/ConnectorContext.java @@ -17,12 +17,9 @@ package org.apache.kafka.connect.connector; -import org.apache.kafka.common.annotation.InterfaceStability; - /** * ConnectorContext allows Connectors to proactively interact with the Kafka Connect runtime. */ -@InterfaceStability.Unstable public interface ConnectorContext { /** * Requests that the runtime reconfigure the Tasks for this source. This should be used to diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java index 850954daa3a49..261134d2b4951 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Task.java @@ -17,8 +17,6 @@ package org.apache.kafka.connect.connector; -import org.apache.kafka.common.annotation.InterfaceStability; - import java.util.Map; /** @@ -34,7 +32,6 @@ * {@link org.apache.kafka.connect.sink.SinkTask}. *

    */ -@InterfaceStability.Unstable public interface Task { /** * Get the version of this task. Usually this should be the same as the corresponding {@link Connector} class's version. diff --git a/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java b/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java index 1202be36aad47..bdddca03cbc96 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/errors/ConnectException.java @@ -18,12 +18,10 @@ package org.apache.kafka.connect.errors; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.annotation.InterfaceStability; /** * ConnectException is the top-level exception type generated by Kafka Connect and connector implementations. */ -@InterfaceStability.Unstable public class ConnectException extends KafkaException { public ConnectException(String s) { diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java index fbe6975f36f08..fd99c93b1ba6d 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkConnector.java @@ -17,13 +17,11 @@ package org.apache.kafka.connect.sink; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.connector.Connector; /** * SinkConnectors implement the Connector interface to send Kafka data to another system. */ -@InterfaceStability.Unstable public abstract class SinkConnector extends Connector { /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java index ad1b2d55e154d..894bf0d29d608 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkRecord.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.sink; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Schema; @@ -30,7 +29,6 @@ * It also includes the {@link TimestampType}, which may be {@link TimestampType#NO_TIMESTAMP_TYPE}, and the relevant * timestamp, which may be {@code null}. */ -@InterfaceStability.Unstable public class SinkRecord extends ConnectRecord { private final long kafkaOffset; private final TimestampType timestampType; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java index 99a26832a86d1..0ea1ecf86b5c0 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTask.java @@ -18,7 +18,6 @@ import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.connector.Task; import java.util.Collection; @@ -52,7 +51,6 @@ * * */ -@InterfaceStability.Unstable public abstract class SinkTask implements Task { /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java index 14f13d1179ac4..a9d2c05a7e39f 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/sink/SinkTaskContext.java @@ -18,7 +18,6 @@ package org.apache.kafka.connect.sink; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.annotation.InterfaceStability; import java.util.Map; import java.util.Set; @@ -26,7 +25,6 @@ /** * Context passed to SinkTasks, allowing them to access utilities in the Kafka Connect runtime. */ -@InterfaceStability.Unstable public interface SinkTaskContext { /** * Reset the consumer offsets for the given topic partitions. SinkTasks should use this if they manage offsets diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java index 2ba5139395348..b67245bcb77d5 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceConnector.java @@ -16,14 +16,12 @@ **/ package org.apache.kafka.connect.source; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.connector.Connector; /** * SourceConnectors implement the connector interface to pull data from another system and send * it to Kafka. */ -@InterfaceStability.Unstable public abstract class SourceConnector extends Connector { } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java index 444979a41dfbf..66d7e89307db9 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceRecord.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.source; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Schema; @@ -41,7 +40,6 @@ * "table_name"} and the sourceOffset as a Long containing the timestamp of the row. *

    */ -@InterfaceStability.Unstable public class SourceRecord extends ConnectRecord { private final Map sourcePartition; private final Map sourceOffset; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java index c08508553dcd9..9caca05191176 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.source; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.connector.Task; import java.util.List; @@ -26,7 +25,6 @@ /** * SourceTask is a Task that pulls records from another system for storage in Kafka. */ -@InterfaceStability.Unstable public abstract class SourceTask implements Task { protected SourceTaskContext context; diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java index 200fa5ff04c1e..674f33c741dd4 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTaskContext.java @@ -16,14 +16,12 @@ **/ package org.apache.kafka.connect.source; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.storage.OffsetStorageReader; /** * SourceTaskContext is provided to SourceTasks to allow them to interact with the underlying * runtime. */ -@InterfaceStability.Unstable public interface SourceTaskContext { /** * Get the OffsetStorageReader for this SourceTask. 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 c2fda2be32e79..763ce5ee09390 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 @@ -17,7 +17,6 @@ package org.apache.kafka.connect.storage; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaAndValue; @@ -28,7 +27,6 @@ * and byte[]. Internally, this likely includes an intermediate step to the format used by the serialization * layer (e.g. JsonNode, GenericRecord, Message). */ -@InterfaceStability.Unstable public interface Converter { /** diff --git a/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java b/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java index 9307c23b28bd2..10e099cab4a1b 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReader.java @@ -17,8 +17,6 @@ package org.apache.kafka.connect.storage; -import org.apache.kafka.common.annotation.InterfaceStability; - import java.util.Collection; import java.util.Map; @@ -33,7 +31,6 @@ * {@link org.apache.kafka.connect.data.Schema} other than Array, Map, and Struct. *

    */ -@InterfaceStability.Unstable public interface OffsetStorageReader { /** * Get the offset for the specified partition. If the data isn't already available locally, this diff --git a/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java b/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java index 35250eb0eefcb..6b3eb935e1c5b 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/util/ConnectorUtils.java @@ -17,8 +17,6 @@ package org.apache.kafka.connect.util; -import org.apache.kafka.common.annotation.InterfaceStability; - import java.util.ArrayList; import java.util.List; @@ -26,7 +24,6 @@ * Utilities that connector implementations might find useful. Contains common building blocks * for writing connectors. */ -@InterfaceStability.Unstable public class ConnectorUtils { /** * Given a list of elements and a target number of groups, generates list of groups of diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index fc957a73c35f8..778673bc8165b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.cli; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; @@ -47,7 +46,6 @@ * instances. *

    */ -@InterfaceStability.Unstable public class ConnectDistributed { private static final Logger log = LoggerFactory.getLogger(ConnectDistributed.class); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index c125a335e0f83..e3be1c5ebb1e7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.cli; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.runtime.Connect; @@ -51,7 +50,6 @@ * fault tolerant by overriding the settings to use file storage for both. *

    */ -@InterfaceStability.Unstable public class ConnectStandalone { private static final Logger log = LoggerFactory.getLogger(ConnectStandalone.class); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index cbcf14c331956..0dcb0b9b6747e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -27,11 +27,13 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; import java.io.ByteArrayOutputStream; @@ -337,6 +339,40 @@ protected Connector getConnector(String connType) { } } + /** + * Checks a given {@link ConfigInfos} for validation error messages and adds an exception + * to the given {@link Callback} if any were found. + * + * @param configInfos configInfos to read Errors from + * @param callback callback to add config error exception to + * @return true if errors were found in the config + */ + protected final boolean maybeAddConfigErrors( + ConfigInfos configInfos, + Callback> callback + ) { + int errors = configInfos.errorCount(); + boolean hasErrors = errors > 0; + if (hasErrors) { + StringBuilder messages = new StringBuilder(); + messages.append("Connector configuration is invalid and contains the following ") + .append(errors).append(" error(s):"); + for (ConfigInfo configInfo : configInfos.values()) { + for (String msg : configInfo.configValue().errors()) { + messages.append('\n').append(msg); + } + } + callback.onCompletion( + new BadRequestException( + messages.append( + "\nYou can also find the above list of errors at the endpoint `/{connectorType}/config/validate`" + ).toString() + ), null + ); + } + return hasErrors; + } + private String trace(Throwable t) { ByteArrayOutputStream output = new ByteArrayOutputStream(); t.printStackTrace(new PrintStream(output)); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index 86f7f23f7a295..219baf0876e8f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.runtime; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.connect.runtime.rest.RestServer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -29,7 +28,6 @@ * This class ties together all the components of a Kafka Connect process (herder, worker, * storage, command interface), managing their lifecycle. */ -@InterfaceStability.Unstable public class Connect { private static final Logger log = LoggerFactory.getLogger(Connect.class); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java index 847f527c43dd9..6fb843e44f88c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/PluginDiscovery.java @@ -107,7 +107,7 @@ private static List> transformationPlugins(Refle Collections.sort(transformationPlugins, new Comparator>() { @Override public int compare(Class a, Class b) { - return a.getCanonicalName().compareTo(b.getCanonicalName()); + return a.getName().compareTo(b.getName()); } }); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index fc3fe7182aa14..249d41a4ae5de 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.runtime; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; @@ -28,7 +27,6 @@ /** * Common base class providing configuration for Kafka Connect workers, whether standalone or distributed. */ -@InterfaceStability.Unstable public class WorkerConfig extends AbstractConfig { public static final String BOOTSTRAP_SERVERS_CONFIG = "bootstrap.servers"; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 25dfc6bedc6b8..47bd0c4c7a76a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -35,10 +35,8 @@ import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.rest.RestServer; -import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; -import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; import org.apache.kafka.connect.sink.SinkConnector; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; @@ -507,10 +505,7 @@ public void putConnectorConfig(final String connName, final Map new Callable() { @Override public Void call() throws Exception { - ConfigInfos validatedConfig = validateConnectorConfig(config); - if (validatedConfig.errorCount() > 0) { - callback.onCompletion(new BadRequestException("Connector configuration is invalid " + - "(use the endpoint `/{connectorType}/config/validate` to get a full list of errors)"), null); + if (maybeAddConfigErrors(validateConnectorConfig(config), callback)) { return null; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index ac13472ea5d1a..f6ff6654aeca3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -38,6 +38,7 @@ import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; +import java.util.Collections; import java.util.LinkedHashMap; import java.util.List; import java.util.Map; @@ -88,7 +89,7 @@ public WorkerGroupMember(DistributedConfig config, this.retryBackoffMs = config.getLong(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG); this.metadata = new Metadata(retryBackoffMs, config.getLong(CommonClientConfigs.METADATA_MAX_AGE_CONFIG)); List addresses = ClientUtils.parseAndValidateAddresses(config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG)); - this.metadata.update(Cluster.bootstrap(addresses), 0); + this.metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), 0); String metricGrpPrefix = "connect"; ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(config.values()); NetworkClient netClient = new NetworkClient( diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java index 0d208022c3cb7..c11a4d3c025a6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java @@ -40,6 +40,7 @@ @Consumes(MediaType.APPLICATION_JSON) public class ConnectorPluginsResource { + private static final String ALIAS_SUFFIX = "Connector"; private final Herder herder; public ConnectorPluginsResource(Herder herder) { @@ -51,8 +52,13 @@ public ConnectorPluginsResource(Herder herder) { public ConfigInfos validateConfigs(final @PathParam("connectorType") String connType, final Map connectorConfig) throws Throwable { String includedConnType = connectorConfig.get(ConnectorConfig.CONNECTOR_CLASS_CONFIG); - if (includedConnType != null && !includedConnType.equals(connType)) - throw new BadRequestException("Included connector type " + includedConnType + " does not match request type " + connType); + if (includedConnType != null + && !normalizedPluginName(includedConnType).endsWith(normalizedPluginName(connType))) { + throw new BadRequestException( + "Included connector type " + includedConnType + " does not match request type " + + connType + ); + } return herder.validateConnectorConfig(connectorConfig); } @@ -62,4 +68,11 @@ public ConfigInfos validateConfigs(final @PathParam("connectorType") String conn public List listConnectorPlugins() { return PluginDiscovery.connectorPlugins(); } + + private String normalizedPluginName(String pluginName) { + // Works for both full and simple class names. In the latter case, it generates the alias. + return pluginName.endsWith(ALIAS_SUFFIX) && pluginName.length() > ALIAS_SUFFIX.length() + ? pluginName.substring(0, pluginName.length() - ALIAS_SUFFIX.length()) + : pluginName; + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index ff09c1cde16ea..6aec9c965e835 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -28,10 +28,8 @@ import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; -import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; -import org.apache.kafka.connect.runtime.rest.errors.BadRequestException; import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.MemoryConfigBackingStore; import org.apache.kafka.connect.storage.MemoryStatusBackingStore; @@ -46,6 +44,7 @@ import java.util.List; import java.util.Map; + /** * Single process, in-memory "herder". Useful for a standalone Kafka Connect process. */ @@ -156,10 +155,7 @@ public synchronized void putConnectorConfig(String connName, boolean allowReplace, final Callback> callback) { try { - ConfigInfos validatedConfig = validateConnectorConfig(config); - if (validatedConfig.errorCount() > 0) { - callback.onCompletion(new BadRequestException("Connector configuration is invalid " + - "(use the endpoint `/{connectorType}/config/validate` to get a full list of errors)"), null); + if (maybeAddConfigErrors(validateConnectorConfig(config), callback)) { return; } 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 new file mode 100644 index 0000000000000..6746042baa743 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/TransformationDoc.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.connect.tools; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.transforms.ExtractField; +import org.apache.kafka.connect.transforms.HoistField; +import org.apache.kafka.connect.transforms.InsertField; +import org.apache.kafka.connect.transforms.MaskField; +import org.apache.kafka.connect.transforms.RegexRouter; +import org.apache.kafka.connect.transforms.ReplaceField; +import org.apache.kafka.connect.transforms.SetSchemaMetadata; +import org.apache.kafka.connect.transforms.TimestampRouter; +import org.apache.kafka.connect.transforms.ValueToKey; + +import java.io.PrintStream; +import java.util.Arrays; +import java.util.List; + +public class TransformationDoc { + + private static final class DocInfo { + final String transformationName; + final String overview; + final ConfigDef configDef; + + private DocInfo(String transformationName, String overview, ConfigDef configDef) { + this.transformationName = transformationName; + this.overview = overview; + this.configDef = configDef; + } + } + + private static final List TRANSFORMATIONS = Arrays.asList( + new DocInfo(InsertField.class.getName(), InsertField.OVERVIEW_DOC, InsertField.CONFIG_DEF), + new DocInfo(ReplaceField.class.getName(), ReplaceField.OVERVIEW_DOC, ReplaceField.CONFIG_DEF), + new DocInfo(MaskField.class.getName(), MaskField.OVERVIEW_DOC, MaskField.CONFIG_DEF), + new DocInfo(ValueToKey.class.getName(), ValueToKey.OVERVIEW_DOC, ValueToKey.CONFIG_DEF), + new DocInfo(HoistField.class.getName(), HoistField.OVERVIEW_DOC, HoistField.CONFIG_DEF), + new DocInfo(ExtractField.class.getName(), ExtractField.OVERVIEW_DOC, ExtractField.CONFIG_DEF), + new DocInfo(SetSchemaMetadata.class.getName(), SetSchemaMetadata.OVERVIEW_DOC, SetSchemaMetadata.CONFIG_DEF), + new DocInfo(TimestampRouter.class.getName(), TimestampRouter.OVERVIEW_DOC, TimestampRouter.CONFIG_DEF), + new DocInfo(RegexRouter.class.getName(), RegexRouter.OVERVIEW_DOC, RegexRouter.CONFIG_DEF) + ); + + private static void printTransformationHtml(PrintStream out, DocInfo docInfo) { + out.println("

    "); + + out.print("
    "); + out.print(docInfo.transformationName); + out.println("
    "); + + out.println(docInfo.overview); + + out.println("

    "); + + out.println(docInfo.configDef.toHtmlTable()); + + out.println("

    "); + } + + private static void printHtml(PrintStream out) throws NoSuchFieldException, IllegalAccessException, InstantiationException { + for (final DocInfo docInfo : TRANSFORMATIONS) { + printTransformationHtml(out, docInfo); + } + } + + public static void main(String... args) throws Exception { + printHtml(System.out); + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java index 45125ccfba9c5..d0fa1c878bad6 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitterTest.java @@ -165,7 +165,7 @@ public void testRemove() throws Exception { EasyMock.expect(task.cancel(eq(false))).andReturn(false); EasyMock.expect(task.isDone()).andReturn(false); EasyMock.expect(task.get()).andThrow(new CancellationException()); - mockLog.trace(EasyMock.anyString(), EasyMock.anyObject()); + mockLog.trace(EasyMock.anyString(), EasyMock.anyObject()); PowerMock.expectLastCall(); PowerMock.replayAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index 92393a1b4fbe5..282e1750dec09 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -91,7 +91,7 @@ public void setup() { this.time = new MockTime(); this.client = new MockClient(time); this.metadata = new Metadata(0, Long.MAX_VALUE); - this.metadata.update(cluster, time.milliseconds()); + this.metadata.update(cluster, Collections.emptySet(), time.milliseconds()); this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index b974559ae0499..b02b8438d9532 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -65,6 +65,8 @@ import java.util.Map; import java.util.Set; +import javax.ws.rs.BadRequestException; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -74,46 +76,61 @@ @PowerMockIgnore("javax.management.*") public class ConnectorPluginsResourceTest { - private static Map props = new HashMap<>(); + private static Map props; + private static Map partialProps = new HashMap<>(); static { - props.put("name", "test"); - props.put("test.string.config", "testString"); - props.put("test.int.config", "1"); - props.put("test.list.config", "a,b"); + partialProps.put("name", "test"); + partialProps.put("test.string.config", "testString"); + partialProps.put("test.int.config", "1"); + partialProps.put("test.list.config", "a,b"); + + props = new HashMap<>(partialProps); + props.put("connector.class", ConnectorPluginsResourceTestConnector.class.getSimpleName()); } private static final ConfigInfos CONFIG_INFOS; - private static final int ERROR_COUNT = 1; + private static final ConfigInfos PARTIAL_CONFIG_INFOS; + private static final int ERROR_COUNT = 0; + private static final int PARTIAL_CONFIG_ERROR_COUNT = 1; static { List configs = new LinkedList<>(); + List partialConfigs = new LinkedList<>(); ConfigDef connectorConfigDef = ConnectorConfig.configDef(); List connectorConfigValues = connectorConfigDef.validate(props); + List partialConnectorConfigValues = connectorConfigDef.validate(partialProps); ConfigInfos result = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), connectorConfigValues, Collections.emptyList()); + ConfigInfos partialResult = AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), connectorConfigDef.configKeys(), partialConnectorConfigValues, Collections.emptyList()); configs.addAll(result.values()); + partialConfigs.addAll(partialResult.values()); ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.emptyList()); ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true); ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); + partialConfigs.add(configInfo); configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.emptyList()); configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); + partialConfigs.add(configInfo); configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.emptyList()); configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); + partialConfigs.add(configInfo); configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, "", "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.emptyList()); configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.emptyList(), true); configInfo = new ConfigInfo(configKeyInfo, configValueInfo); configs.add(configInfo); + partialConfigs.add(configInfo); CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), ERROR_COUNT, Collections.singletonList("Test"), configs); + PARTIAL_CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), PARTIAL_CONFIG_ERROR_COUNT, Collections.singletonList("Test"), partialConfigs); } @Mock @@ -128,7 +145,55 @@ public void setUp() throws NoSuchMethodException { } @Test - public void testValidateConfig() throws Throwable { + public void testValidateConfigWithSingleErrorDueToMissingConnectorClassname() throws Throwable { + herder.validateConnectorConfig(EasyMock.eq(partialProps)); + + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public ConfigInfos answer() { + ConfigDef connectorConfigDef = ConnectorConfig.configDef(); + List connectorConfigValues = connectorConfigDef.validate(partialProps); + + Connector connector = new ConnectorPluginsResourceTestConnector(); + Config config = connector.validate(partialProps); + ConfigDef configDef = connector.config(); + Map configKeys = configDef.configKeys(); + List configValues = config.configValues(); + + Map resultConfigKeys = new HashMap<>(configKeys); + resultConfigKeys.putAll(connectorConfigDef.configKeys()); + configValues.addAll(connectorConfigValues); + + return AbstractHerder.generateResult( + ConnectorPluginsResourceTestConnector.class.getName(), + resultConfigKeys, + configValues, + Collections.singletonList("Test") + ); + } + }); + + PowerMock.replayAll(); + + // This call to validateConfigs does not throw a BadRequestException because we've mocked + // validateConnectorConfig. + ConfigInfos configInfos = connectorPluginsResource.validateConfigs( + ConnectorPluginsResourceTestConnector.class.getSimpleName(), + partialProps + ); + assertEquals(PARTIAL_CONFIG_INFOS.name(), configInfos.name()); + assertEquals(PARTIAL_CONFIG_INFOS.errorCount(), configInfos.errorCount()); + assertEquals(PARTIAL_CONFIG_INFOS.groups(), configInfos.groups()); + assertEquals( + new HashSet<>(PARTIAL_CONFIG_INFOS.values()), + new HashSet<>(configInfos.values()) + ); + + PowerMock.verifyAll(); + } + + @Test + public void testValidateConfigWithSimpleName() throws Throwable { herder.validateConnectorConfig(EasyMock.eq(props)); PowerMock.expectLastCall().andAnswer(new IAnswer() { @@ -147,20 +212,150 @@ public ConfigInfos answer() { resultConfigKeys.putAll(connectorConfigDef.configKeys()); configValues.addAll(connectorConfigValues); - return AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), resultConfigKeys, configValues, Collections.singletonList("Test")); + return AbstractHerder.generateResult( + ConnectorPluginsResourceTestConnector.class.getName(), + resultConfigKeys, + configValues, + Collections.singletonList("Test") + ); } }); + PowerMock.replayAll(); - ConfigInfos configInfos = connectorPluginsResource.validateConfigs(ConnectorPluginsResourceTestConnector.class.getName(), props); + // make a request to connector-plugins resource using just the simple class name. + ConfigInfos configInfos = connectorPluginsResource.validateConfigs( + ConnectorPluginsResourceTestConnector.class.getSimpleName(), + props + ); assertEquals(CONFIG_INFOS.name(), configInfos.name()); - assertEquals(CONFIG_INFOS.errorCount(), configInfos.errorCount()); + assertEquals(0, configInfos.errorCount()); assertEquals(CONFIG_INFOS.groups(), configInfos.groups()); assertEquals(new HashSet<>(CONFIG_INFOS.values()), new HashSet<>(configInfos.values())); PowerMock.verifyAll(); } + @Test + public void testValidateConfigWithAlias() throws Throwable { + herder.validateConnectorConfig(EasyMock.eq(props)); + + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public ConfigInfos answer() { + ConfigDef connectorConfigDef = ConnectorConfig.configDef(); + List connectorConfigValues = connectorConfigDef.validate(props); + + Connector connector = new ConnectorPluginsResourceTestConnector(); + Config config = connector.validate(props); + ConfigDef configDef = connector.config(); + Map configKeys = configDef.configKeys(); + List configValues = config.configValues(); + + Map resultConfigKeys = new HashMap<>(configKeys); + resultConfigKeys.putAll(connectorConfigDef.configKeys()); + configValues.addAll(connectorConfigValues); + + return AbstractHerder.generateResult( + ConnectorPluginsResourceTestConnector.class.getName(), + resultConfigKeys, + configValues, + Collections.singletonList("Test") + ); + } + }); + + PowerMock.replayAll(); + + // make a request to connector-plugins resource using a valid alias. + ConfigInfos configInfos = connectorPluginsResource.validateConfigs( + "ConnectorPluginsResourceTest", + props + ); + assertEquals(CONFIG_INFOS.name(), configInfos.name()); + assertEquals(0, configInfos.errorCount()); + assertEquals(CONFIG_INFOS.groups(), configInfos.groups()); + assertEquals(new HashSet<>(CONFIG_INFOS.values()), new HashSet<>(configInfos.values())); + + PowerMock.verifyAll(); + } + + @Test(expected = BadRequestException.class) + public void testValidateConfigWithNonExistentName() throws Throwable { + herder.validateConnectorConfig(EasyMock.eq(props)); + + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public ConfigInfos answer() { + ConfigDef connectorConfigDef = ConnectorConfig.configDef(); + List connectorConfigValues = connectorConfigDef.validate(props); + + Connector connector = new ConnectorPluginsResourceTestConnector(); + Config config = connector.validate(props); + ConfigDef configDef = connector.config(); + Map configKeys = configDef.configKeys(); + List configValues = config.configValues(); + + Map resultConfigKeys = new HashMap<>(configKeys); + resultConfigKeys.putAll(connectorConfigDef.configKeys()); + configValues.addAll(connectorConfigValues); + + return AbstractHerder.generateResult( + ConnectorPluginsResourceTestConnector.class.getName(), + resultConfigKeys, + configValues, + Collections.singletonList("Test") + ); + } + }); + + PowerMock.replayAll(); + + // make a request to connector-plugins resource using a non-loaded connector with the same + // simple name but different package. + String customClassname = "com.custom.package." + + ConnectorPluginsResourceTestConnector.class.getSimpleName(); + connectorPluginsResource.validateConfigs(customClassname, props); + + PowerMock.verifyAll(); + } + + @Test(expected = BadRequestException.class) + public void testValidateConfigWithNonExistentAlias() throws Throwable { + herder.validateConnectorConfig(EasyMock.eq(props)); + + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public ConfigInfos answer() { + ConfigDef connectorConfigDef = ConnectorConfig.configDef(); + List connectorConfigValues = connectorConfigDef.validate(props); + + Connector connector = new ConnectorPluginsResourceTestConnector(); + Config config = connector.validate(props); + ConfigDef configDef = connector.config(); + Map configKeys = configDef.configKeys(); + List configValues = config.configValues(); + + Map resultConfigKeys = new HashMap<>(configKeys); + resultConfigKeys.putAll(connectorConfigDef.configKeys()); + configValues.addAll(connectorConfigValues); + + return AbstractHerder.generateResult( + ConnectorPluginsResourceTestConnector.class.getName(), + resultConfigKeys, + configValues, + Collections.singletonList("Test") + ); + } + }); + + PowerMock.replayAll(); + + connectorPluginsResource.validateConfigs("ConnectorPluginsTest", props); + + PowerMock.verifyAll(); + } + @Test public void testListConnectorPlugins() { Set connectorPlugins = new HashSet<>(connectorPluginsResource.listConnectorPlugins()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index e05aa41299b92..9092f523fe898 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -57,6 +57,7 @@ import org.powermock.api.easymock.annotation.Mock; import org.powermock.modules.junit4.PowerMockRunner; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -485,6 +486,46 @@ public void testPutTaskConfigs() { PowerMock.verifyAll(); } + @Test + public void testCorruptConfig() { + Map config = new HashMap<>(); + config.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_NAME); + config.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, BogusSinkConnector.class.getName()); + Connector connectorMock = PowerMock.createMock(Connector.class); + String error = "This is an error in your config!"; + List errors = new ArrayList<>(singletonList(error)); + String key = "foo.invalid.key"; + EasyMock.expect(connectorMock.validate(config)).andReturn( + new Config( + Arrays.asList(new ConfigValue(key, null, Collections.emptyList(), errors)) + ) + ); + ConfigDef configDef = new ConfigDef(); + configDef.define(key, ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, ""); + EasyMock.expect(connectorMock.config()).andStubReturn(configDef); + ConnectorFactory connectorFactoryMock = PowerMock.createMock(ConnectorFactory.class); + EasyMock.expect(worker.getConnectorFactory()).andStubReturn(connectorFactoryMock); + EasyMock.expect(connectorFactoryMock.newConnector(EasyMock.anyString())) + .andReturn(connectorMock); + Callback> callback = PowerMock.createMock(Callback.class); + Capture capture = Capture.newInstance(); + callback.onCompletion( + EasyMock.capture(capture), EasyMock.isNull(Herder.Created.class) + ); + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, config, true, callback); + assertEquals( + capture.getValue().getMessage(), + "Connector configuration is invalid and contains the following 1 error(s):\n" + + error + "\n" + + "You can also find the above list of errors at the endpoint `/{connectorType}/config/validate`" + ); + + PowerMock.verifyAll(); + } + private void expectAdd(SourceSink sourceSink) throws Exception { Map connectorProps = connectorConfig(sourceSink); diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java new file mode 100644 index 0000000000000..5e934ca39d67c --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ExtractField.java @@ -0,0 +1,114 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class ExtractField> implements Transformation { + + public static final String OVERVIEW_DOC = + "Extract the specified field from a Struct when schema present, or a Map in the case of schemaless data." + + "

    Use the concrete transformation type designed for the record key (" + Key.class.getName() + ") " + + "or value (" + Value.class.getName() + ")."; + + private static final String FIELD_CONFIG = "field"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, "Field name to extract."); + + private static final String PURPOSE = "field extraction"; + + private String fieldName; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + fieldName = config.getString(FIELD_CONFIG); + } + + @Override + public R apply(R record) { + final Schema schema = operatingSchema(record); + if (schema == null) { + final Map value = requireMap(operatingValue(record), PURPOSE); + return newRecord(record, null, value.get(fieldName)); + } else { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + return newRecord(record, schema.field(fieldName).schema(), value.get(fieldName)); + } + } + + @Override + public void close() { + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); + + public static class Key> extends ExtractField { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + } + + public static class Value> extends ExtractField { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); + } + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java similarity index 73% rename from connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java rename to connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java index c2726ca84378b..9ae1c8115c9f5 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistToStruct.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/HoistField.java @@ -27,15 +27,21 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import java.util.Collections; import java.util.Map; -public abstract class HoistToStruct> implements Transformation { +public abstract class HoistField> implements Transformation { - public static final String FIELD_CONFIG = "field"; + public static final String OVERVIEW_DOC = + "Wrap data using the specified field name in a Struct when schema present, or a Map in the case of schemaless data." + + "

    Use the concrete transformation type designed for the record key (" + Key.class.getName() + ") " + + "or value (" + Value.class.getName() + ")."; - private static final ConfigDef CONFIG_DEF = new ConfigDef() + private static final String FIELD_CONFIG = "field"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() .define(FIELD_CONFIG, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.MEDIUM, - "Field name for the single field that will be created in the resulting Struct."); + "Field name for the single field that will be created in the resulting Struct or Map."); private Cache schemaUpdateCache; @@ -53,15 +59,19 @@ public R apply(R record) { final Schema schema = operatingSchema(record); final Object value = operatingValue(record); - Schema updatedSchema = schemaUpdateCache.get(schema); - if (updatedSchema == null) { - updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build(); - schemaUpdateCache.put(schema, updatedSchema); - } + if (schema == null) { + return newRecord(record, null, Collections.singletonMap(fieldName, value)); + } else { + Schema updatedSchema = schemaUpdateCache.get(schema); + if (updatedSchema == null) { + updatedSchema = SchemaBuilder.struct().field(fieldName, schema).build(); + schemaUpdateCache.put(schema, updatedSchema); + } - final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value); + final Struct updatedValue = new Struct(updatedSchema).put(fieldName, value); - return newRecord(record, updatedSchema, updatedValue); + return newRecord(record, updatedSchema, updatedValue); + } } @Override @@ -80,11 +90,7 @@ public ConfigDef config() { protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); - /** - * Wraps the record key in a {@link org.apache.kafka.connect.data.Struct} with specified field name. - */ - public static class Key> extends HoistToStruct { - + public static class Key> extends HoistField { @Override protected Schema operatingSchema(R record) { return record.keySchema(); @@ -99,14 +105,9 @@ protected Object operatingValue(R record) { protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); } - } - /** - * Wraps the record value in a {@link org.apache.kafka.connect.data.Struct} with specified field name. - */ - public static class Value> extends HoistToStruct { - + public static class Value> extends HoistField { @Override protected Schema operatingSchema(R record) { return record.valueSchema(); @@ -121,7 +122,6 @@ protected Object operatingValue(R record) { protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); } - } } diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java index d67fea0893237..54b6c1450504e 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/InsertField.java @@ -21,23 +21,32 @@ import org.apache.kafka.common.cache.LRUCache; import org.apache.kafka.common.cache.SynchronizedCache; import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.connect.connector.ConnectRecord; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; import org.apache.kafka.connect.data.SchemaBuilder; import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.data.Timestamp; -import org.apache.kafka.connect.errors.DataException; -import org.apache.kafka.connect.sink.SinkRecord; import org.apache.kafka.connect.transforms.util.SimpleConfig; +import org.apache.kafka.connect.transforms.util.SchemaUtil; import java.util.Date; import java.util.HashMap; import java.util.Map; +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireSinkRecord; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + public abstract class InsertField> implements Transformation { - public interface Keys { + public static final String OVERVIEW_DOC = + "Insert field(s) using attributes from the record metadata or a configured static value." + + "

    Use the concrete transformation type designed for the record key (" + Key.class.getName() + ") " + + "or value (" + Value.class.getName() + ")."; + + private interface ConfigName { String TOPIC_FIELD = "topic.field"; String PARTITION_FIELD = "partition.field"; String OFFSET_FIELD = "offset.field"; @@ -46,22 +55,24 @@ public interface Keys { String STATIC_VALUE = "static.value"; } - private static final String OPTIONALITY_DOC = "Suffix with '!' to make this a required field, or '?' to keep it optional (the default)."; - - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(Keys.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka topic.\n" + OPTIONALITY_DOC) - .define(Keys.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka partition.\n" + OPTIONALITY_DOC) - .define(Keys.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for Kafka offset - only applicable to sink connectors.\n" + OPTIONALITY_DOC) - .define(Keys.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for record timestamp.\n" + OPTIONALITY_DOC) - .define(Keys.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, - "Field name for static data field.\n" + OPTIONALITY_DOC) - .define(Keys.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + private static final String OPTIONALITY_DOC = "Suffix with ! to make this a required field, or ? to keep it optional (the default)."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.TOPIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka topic. " + OPTIONALITY_DOC) + .define(ConfigName.PARTITION_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka partition. " + OPTIONALITY_DOC) + .define(ConfigName.OFFSET_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for Kafka offset - only applicable to sink connectors.
    " + OPTIONALITY_DOC) + .define(ConfigName.TIMESTAMP_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for record timestamp. " + OPTIONALITY_DOC) + .define(ConfigName.STATIC_FIELD, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, + "Field name for static data field. " + OPTIONALITY_DOC) + .define(ConfigName.STATIC_VALUE, ConfigDef.Type.STRING, null, ConfigDef.Importance.MEDIUM, "Static field value, if field name configured."); + private static final String PURPOSE = "field insertion"; + private static final Schema OPTIONAL_TIMESTAMP_SCHEMA = Timestamp.builder().optional().build(); private static final class InsertionSpec { @@ -91,46 +102,42 @@ public static InsertionSpec parse(String spec) { private InsertionSpec timestampField; private InsertionSpec staticField; private String staticValue; - private boolean applicable; private Cache schemaUpdateCache; @Override public void configure(Map props) { final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); - topicField = InsertionSpec.parse(config.getString(Keys.TOPIC_FIELD)); - partitionField = InsertionSpec.parse(config.getString(Keys.PARTITION_FIELD)); - offsetField = InsertionSpec.parse(config.getString(Keys.OFFSET_FIELD)); - timestampField = InsertionSpec.parse(config.getString(Keys.TIMESTAMP_FIELD)); - staticField = InsertionSpec.parse(config.getString(Keys.STATIC_FIELD)); - staticValue = config.getString(Keys.STATIC_VALUE); - applicable = topicField != null || partitionField != null || offsetField != null || timestampField != null; + topicField = InsertionSpec.parse(config.getString(ConfigName.TOPIC_FIELD)); + partitionField = InsertionSpec.parse(config.getString(ConfigName.PARTITION_FIELD)); + offsetField = InsertionSpec.parse(config.getString(ConfigName.OFFSET_FIELD)); + timestampField = InsertionSpec.parse(config.getString(ConfigName.TIMESTAMP_FIELD)); + staticField = InsertionSpec.parse(config.getString(ConfigName.STATIC_FIELD)); + staticValue = config.getString(ConfigName.STATIC_VALUE); + + if (topicField == null && partitionField == null && offsetField == null && timestampField == null && staticField == null) { + throw new ConfigException("No field insertion configured"); + } + + if (staticField != null && staticValue == null) { + throw new ConfigException(ConfigName.STATIC_VALUE, null, "No value specified for static field: " + staticField); + } schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16)); } @Override public R apply(R record) { - if (!applicable) return record; - - final Schema schema = operatingSchema(record); - final Object value = operatingValue(record); - - if (value == null) - throw new DataException("null value"); - - if (schema == null) { - if (!(value instanceof Map)) - throw new DataException("Can only operate on Map value in schemaless mode: " + value.getClass().getName()); - return applySchemaless(record, (Map) value); + if (operatingSchema(record) == null) { + return applySchemaless(record); } else { - if (schema.type() != Schema.Type.STRUCT) - throw new DataException("Can only operate on Struct types: " + value.getClass().getName()); - return applyWithSchema(record, schema, (Struct) value); + return applyWithSchema(record); } } - private R applySchemaless(R record, Map value) { + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + final Map updatedValue = new HashMap<>(value); if (topicField != null) { @@ -140,9 +147,7 @@ private R applySchemaless(R record, Map value) { updatedValue.put(partitionField.name, record.kafkaPartition()); } if (offsetField != null) { - if (!(record instanceof SinkRecord)) - throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass()); - updatedValue.put(offsetField.name, ((SinkRecord) record).kafkaOffset()); + updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset()); } if (timestampField != null && record.timestamp() != null) { updatedValue.put(timestampField.name, record.timestamp()); @@ -150,36 +155,46 @@ private R applySchemaless(R record, Map value) { if (staticField != null && staticValue != null) { updatedValue.put(staticField.name, staticValue); } + return newRecord(record, null, updatedValue); } - private R applyWithSchema(R record, Schema schema, Struct value) { - Schema updatedSchema = schemaUpdateCache.get(schema); + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + + Schema updatedSchema = schemaUpdateCache.get(value.schema()); if (updatedSchema == null) { - updatedSchema = makeUpdatedSchema(schema); - schemaUpdateCache.put(schema, updatedSchema); + updatedSchema = makeUpdatedSchema(value.schema()); + schemaUpdateCache.put(value.schema(), updatedSchema); } final Struct updatedValue = new Struct(updatedSchema); - copyFields(value, updatedValue); + for (Field field : value.schema().fields()) { + updatedValue.put(field.name(), value.get(field)); + } - insertFields(record, updatedValue); + if (topicField != null) { + updatedValue.put(topicField.name, record.topic()); + } + if (partitionField != null && record.kafkaPartition() != null) { + updatedValue.put(partitionField.name, record.kafkaPartition()); + } + if (offsetField != null) { + updatedValue.put(offsetField.name, requireSinkRecord(record, PURPOSE).kafkaOffset()); + } + if (timestampField != null && record.timestamp() != null) { + updatedValue.put(timestampField.name, new Date(record.timestamp())); + } + if (staticField != null && staticValue != null) { + updatedValue.put(staticField.name, staticValue); + } return newRecord(record, updatedSchema, updatedValue); } private Schema makeUpdatedSchema(Schema schema) { - final SchemaBuilder builder = SchemaBuilder.struct(); - - builder.name(schema.name()); - builder.version(schema.version()); - builder.doc(schema.doc()); - - final Map params = schema.parameters(); - if (params != null) { - builder.parameters(params); - } + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); for (Field field : schema.fields()) { builder.field(field.name(), field.schema()); @@ -204,33 +219,6 @@ private Schema makeUpdatedSchema(Schema schema) { return builder.build(); } - private void copyFields(Struct value, Struct updatedValue) { - for (Field field : value.schema().fields()) { - updatedValue.put(field.name(), value.get(field)); - } - } - - private void insertFields(R record, Struct value) { - if (topicField != null) { - value.put(topicField.name, record.topic()); - } - if (partitionField != null && record.kafkaPartition() != null) { - value.put(partitionField.name, record.kafkaPartition()); - } - if (offsetField != null) { - if (!(record instanceof SinkRecord)) { - throw new DataException("Offset insertion is only supported for sink connectors, record is of type: " + record.getClass()); - } - value.put(offsetField.name, ((SinkRecord) record).kafkaOffset()); - } - if (timestampField != null && record.timestamp() != null) { - value.put(timestampField.name, new Date(record.timestamp())); - } - if (staticField != null && staticValue != null) { - value.put(staticField.name, staticValue); - } - } - @Override public void close() { schemaUpdateCache = null; @@ -247,10 +235,6 @@ public ConfigDef config() { protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); - /** - * This transformation allows inserting configured attributes of the record metadata as fields in the record key. - * It also allows adding a static data field. - */ public static class Key> extends InsertField { @Override @@ -270,10 +254,6 @@ protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { } - /** - * This transformation allows inserting configured attributes of the record metadata as fields in the record value. - * It also allows adding a static data field. - */ public static class Value> extends InsertField { @Override diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java new file mode 100644 index 0000000000000..ae8cf864b2f14 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/MaskField.java @@ -0,0 +1,172 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Collections; +import java.util.Date; +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.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class MaskField> implements Transformation { + + public static final String OVERVIEW_DOC = + "Mask specified fields with a valid null value for the field type (i.e. 0, false, empty string, and so on)." + + "

    Use the concrete transformation type designed for the record key (" + Key.class.getName() + ") " + + "or value (" + Value.class.getName() + ")."; + + private static final String FIELDS_CONFIG = "fields"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, "Names of fields to mask."); + + private static final String PURPOSE = "mask fields"; + + private static final Map, Object> PRIMITIVE_VALUE_MAPPING = new HashMap<>(); + + static { + PRIMITIVE_VALUE_MAPPING.put(Boolean.class, Boolean.FALSE); + PRIMITIVE_VALUE_MAPPING.put(Byte.class, (byte) 0); + PRIMITIVE_VALUE_MAPPING.put(Short.class, (short) 0); + PRIMITIVE_VALUE_MAPPING.put(Integer.class, 0); + PRIMITIVE_VALUE_MAPPING.put(Long.class, 0L); + PRIMITIVE_VALUE_MAPPING.put(Float.class, 0f); + PRIMITIVE_VALUE_MAPPING.put(Double.class, 0d); + PRIMITIVE_VALUE_MAPPING.put(BigInteger.class, BigInteger.ZERO); + PRIMITIVE_VALUE_MAPPING.put(BigDecimal.class, BigDecimal.ZERO); + PRIMITIVE_VALUE_MAPPING.put(Date.class, new Date(0)); + PRIMITIVE_VALUE_MAPPING.put(String.class, ""); + } + + private Set maskedFields; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + maskedFields = new HashSet<>(config.getList(FIELDS_CONFIG)); + } + + @Override + public R apply(R record) { + if (operatingSchema(record) == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + final HashMap updatedValue = new HashMap<>(value); + for (String field : maskedFields) { + updatedValue.put(field, masked(value.get(field))); + } + return newRecord(record, updatedValue); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + final Struct updatedValue = new Struct(value.schema()); + for (Field field : value.schema().fields()) { + final Object origFieldValue = value.get(field); + updatedValue.put(field, maskedFields.contains(field.name()) ? masked(origFieldValue) : origFieldValue); + } + return newRecord(record, updatedValue); + } + + private static Object masked(Object value) { + if (value == null) + return null; + Object maskedValue = PRIMITIVE_VALUE_MAPPING.get(value.getClass()); + if (maskedValue == null) { + if (value instanceof List) + maskedValue = Collections.emptyList(); + else if (value instanceof Map) + maskedValue = Collections.emptyMap(); + else + throw new DataException("Cannot mask value of type: " + value.getClass()); + } + return maskedValue; + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R base, Object value); + + public static final class Key> extends MaskField { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + } + + public static final class Value> extends MaskField { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), updatedValue, record.timestamp()); + } + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java new file mode 100644 index 0000000000000..f16560e3c5748 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/RegexRouter.java @@ -0,0 +1,75 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.transforms.util.RegexValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class RegexRouter> implements Transformation { + + public static final String OVERVIEW_DOC = "Update the record topic using the configured regular expression and replacement string." + + "

    Under the hood, the regex is compiled to a java.util.regex.Pattern. " + + "If the pattern matches the input topic, java.util.regex.Matcher#replaceFirst() is used with the replacement string to obtain the new topic."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.REGEX, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, new RegexValidator(), ConfigDef.Importance.HIGH, + "Regular expression to use for matching.") + .define(ConfigName.REPLACEMENT, ConfigDef.Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ConfigDef.Importance.HIGH, + "Replacement string."); + + private interface ConfigName { + String REGEX = "regex"; + String REPLACEMENT = "replacement"; + } + + private Pattern regex; + private String replacement; + + @Override + public void configure(Map props) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); + regex = Pattern.compile(config.getString(ConfigName.REGEX)); + replacement = config.getString(ConfigName.REPLACEMENT); + } + + @Override + public R apply(R record) { + final Matcher matcher = regex.matcher(record.topic()); + if (matcher.matches()) { + final String topic = matcher.replaceFirst(replacement); + return record.newRecord(topic, record.kafkaPartition(), record.keySchema(), record.key(), record.valueSchema(), record.value(), record.timestamp()); + } + return record; + } + + @Override + public void close() { + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java new file mode 100644 index 0000000000000..ab71a69f9169c --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ReplaceField.java @@ -0,0 +1,232 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.SchemaUtil; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public abstract class ReplaceField> implements Transformation { + + public static final String OVERVIEW_DOC = "Filter or rename fields." + + "

    Use the concrete transformation type designed for the record key (" + Key.class.getName() + ") " + + "or value (" + Value.class.getName() + ")."; + + interface ConfigName { + String BLACKLIST = "blacklist"; + String WHITELIST = "whitelist"; + String RENAME = "renames"; + } + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.BLACKLIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM, + "Fields to exclude. This takes precedence over the whitelist.") + .define(ConfigName.WHITELIST, ConfigDef.Type.LIST, Collections.emptyList(), ConfigDef.Importance.MEDIUM, + "Fields to include. If specified, only these fields will be used.") + .define(ConfigName.RENAME, ConfigDef.Type.LIST, Collections.emptyList(), new ConfigDef.Validator() { + @Override + public void ensureValid(String name, Object value) { + parseRenameMappings((List) value); + } + + @Override + public String toString() { + return "list of colon-delimited pairs, e.g. foo:bar,abc:xyz"; + } + }, ConfigDef.Importance.MEDIUM, "Field rename mappings."); + + private static final String PURPOSE = "field replacement"; + + private List blacklist; + private List whitelist; + private Map renames; + private Map reverseRenames; + + private Cache schemaUpdateCache; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + blacklist = config.getList(ConfigName.BLACKLIST); + whitelist = config.getList(ConfigName.WHITELIST); + renames = parseRenameMappings(config.getList(ConfigName.RENAME)); + reverseRenames = invert(renames); + + schemaUpdateCache = new SynchronizedCache<>(new LRUCache(16)); + } + + static Map parseRenameMappings(List mappings) { + final Map m = new HashMap<>(); + for (String mapping : mappings) { + final String[] parts = mapping.split(":"); + if (parts.length != 2) { + throw new ConfigException(ConfigName.RENAME, mappings, "Invalid rename mapping: " + mapping); + } + m.put(parts[0], parts[1]); + } + return m; + } + + static Map invert(Map source) { + final Map m = new HashMap<>(); + for (Map.Entry e : source.entrySet()) { + m.put(e.getValue(), e.getKey()); + } + return m; + } + + boolean filter(String fieldName) { + return !blacklist.contains(fieldName) && (whitelist.isEmpty() || whitelist.contains(fieldName)); + } + + String renamed(String fieldName) { + final String mapping = renames.get(fieldName); + return mapping == null ? fieldName : mapping; + } + + String reverseRenamed(String fieldName) { + final String mapping = reverseRenames.get(fieldName); + return mapping == null ? fieldName : mapping; + } + + @Override + public R apply(R record) { + if (operatingSchema(record) == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(operatingValue(record), PURPOSE); + + final Map updatedValue = new HashMap<>(value.size()); + + for (Map.Entry e : value.entrySet()) { + final String fieldName = e.getKey(); + if (filter(fieldName)) { + final Object fieldValue = e.getValue(); + updatedValue.put(renamed(fieldName), fieldValue); + } + } + + return newRecord(record, null, updatedValue); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(operatingValue(record), PURPOSE); + + Schema updatedSchema = schemaUpdateCache.get(value.schema()); + if (updatedSchema == null) { + updatedSchema = makeUpdatedSchema(value.schema()); + schemaUpdateCache.put(value.schema(), updatedSchema); + } + + final Struct updatedValue = new Struct(updatedSchema); + + for (Field field : updatedSchema.fields()) { + final Object fieldValue = value.get(reverseRenamed(field.name())); + updatedValue.put(field.name(), fieldValue); + } + + return newRecord(record, updatedSchema, updatedValue); + } + + private Schema makeUpdatedSchema(Schema schema) { + final SchemaBuilder builder = SchemaUtil.copySchemaBasics(schema, SchemaBuilder.struct()); + for (Field field : schema.fields()) { + if (filter(field.name())) { + builder.field(renamed(field.name()), field.schema()); + } + } + return builder.build(); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + schemaUpdateCache = null; + } + + protected abstract Schema operatingSchema(R record); + + protected abstract Object operatingValue(R record); + + protected abstract R newRecord(R record, Schema updatedSchema, Object updatedValue); + + public static class Key> extends ReplaceField { + + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.key(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, updatedValue, record.valueSchema(), record.value(), record.timestamp()); + } + + } + + public static class Value> extends ReplaceField { + + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected Object operatingValue(R record) { + return record.value(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema, Object updatedValue) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, updatedValue, record.timestamp()); + } + + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java new file mode 100644 index 0000000000000..017e5d4455e04 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/SetSchemaMetadata.java @@ -0,0 +1,124 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.ConnectSchema; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireSchema; + +public abstract class SetSchemaMetadata> implements Transformation { + + public static final String OVERVIEW_DOC = + "Set the schema name, version or both on the record's key (" + Key.class.getName() + ")" + + " or value (" + Value.class.getName() + ") schema."; + + private interface ConfigName { + String SCHEMA_NAME = "schema.name"; + String SCHEMA_VERSION = "schema.version"; + } + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.SCHEMA_NAME, ConfigDef.Type.STRING, null, ConfigDef.Importance.HIGH, "Schema name to set.") + .define(ConfigName.SCHEMA_VERSION, ConfigDef.Type.INT, null, ConfigDef.Importance.HIGH, "Schema version to set."); + + private String schemaName; + private Integer schemaVersion; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + schemaName = config.getString(ConfigName.SCHEMA_NAME); + schemaVersion = config.getInt(ConfigName.SCHEMA_VERSION); + + if (schemaName == null && schemaVersion == null) { + throw new ConfigException("Neither schema name nor version configured"); + } + } + + @Override + public R apply(R record) { + final Schema schema = operatingSchema(record); + requireSchema(schema, "updating schema metadata"); + final boolean isArray = schema.type() == Schema.Type.ARRAY; + final boolean isMap = schema.type() == Schema.Type.MAP; + final Schema updatedSchema = new ConnectSchema( + schema.type(), + schema.isOptional(), + schema.defaultValue(), + schemaName != null ? schemaName : schema.name(), + schemaVersion != null ? schemaVersion : schema.version(), + schema.doc(), + schema.parameters(), + schema.fields(), + isMap ? schema.keySchema() : null, + isMap || isArray ? schema.valueSchema() : null + ); + return newRecord(record, updatedSchema); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + } + + protected abstract Schema operatingSchema(R record); + + protected abstract R newRecord(R record, Schema updatedSchema); + + /** + * Set the schema name, version or both on the record's key schema. + */ + public static class Key> extends SetSchemaMetadata { + @Override + protected Schema operatingSchema(R record) { + return record.keySchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema) { + return record.newRecord(record.topic(), record.kafkaPartition(), updatedSchema, record.key(), record.valueSchema(), record.value(), record.timestamp()); + } + } + + /** + * Set the schema name, version or both on the record's value schema. + */ + public static class Value> extends SetSchemaMetadata { + @Override + protected Schema operatingSchema(R record) { + return record.valueSchema(); + } + + @Override + protected R newRecord(R record, Schema updatedSchema) { + return record.newRecord(record.topic(), record.kafkaPartition(), record.keySchema(), record.key(), updatedSchema, record.value(), record.timestamp()); + } + } + +} \ No newline at end of file diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java index 1dd5345a6091a..f917a8d65f09a 100644 --- a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/TimestampRouter.java @@ -27,25 +27,25 @@ import java.util.Map; import java.util.TimeZone; -/** - * This transformation facilitates updating the record's topic field as a function of the original topic value and the record timestamp. - *

    - * It is mainly useful for sink connectors, since the topic field is often used to determine the equivalent entity name in the destination system - * (e.g. database table or search index name). - */ public class TimestampRouter> implements Transformation { - public interface Keys { + public static final String OVERVIEW_DOC = + "Update the record's topic field as a function of the original topic value and the record timestamp." + + "

    " + + "This is mainly useful for sink connectors, since the topic field is often used to determine the equivalent entity name in the destination system" + + "(e.g. database table or search index name)."; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(ConfigName.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH, + "Format string which can contain ${topic} and ${timestamp} as placeholders for the topic and timestamp, respectively.") + .define(ConfigName.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH, + "Format string for the timestamp that is compatible with java.text.SimpleDateFormat."); + + private interface ConfigName { String TOPIC_FORMAT = "topic.format"; String TIMESTAMP_FORMAT = "timestamp.format"; } - private static final ConfigDef CONFIG_DEF = new ConfigDef() - .define(Keys.TOPIC_FORMAT, ConfigDef.Type.STRING, "${topic}-${timestamp}", ConfigDef.Importance.HIGH, - "Format string which can contain ``${topic}`` and ``${timestamp}`` as placeholders for the topic and timestamp, respectively.") - .define(Keys.TIMESTAMP_FORMAT, ConfigDef.Type.STRING, "yyyyMMdd", ConfigDef.Importance.HIGH, - "Format string for the timestamp that is compatible with java.text.SimpleDateFormat."); - private String topicFormat; private ThreadLocal timestampFormat; @@ -53,9 +53,9 @@ public interface Keys { public void configure(Map props) { final SimpleConfig config = new SimpleConfig(CONFIG_DEF, props); - topicFormat = config.getString(Keys.TOPIC_FORMAT); + topicFormat = config.getString(ConfigName.TOPIC_FORMAT); - final String timestampFormatStr = config.getString(Keys.TIMESTAMP_FORMAT); + final String timestampFormatStr = config.getString(ConfigName.TIMESTAMP_FORMAT); timestampFormat = new ThreadLocal() { @Override protected SimpleDateFormat initialValue() { diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java new file mode 100644 index 0000000000000..504da541f6574 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/ValueToKey.java @@ -0,0 +1,111 @@ +/** + * 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.transforms; + +import org.apache.kafka.common.cache.Cache; +import org.apache.kafka.common.cache.LRUCache; +import org.apache.kafka.common.cache.SynchronizedCache; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.transforms.util.NonEmptyListValidator; +import org.apache.kafka.connect.transforms.util.SimpleConfig; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.kafka.connect.transforms.util.Requirements.requireMap; +import static org.apache.kafka.connect.transforms.util.Requirements.requireStruct; + +public class ValueToKey> implements Transformation { + + public static final String OVERVIEW_DOC = "Replace the record key with a new key formed from a subset of fields in the record value."; + + public static final String FIELDS_CONFIG = "fields"; + + public static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FIELDS_CONFIG, ConfigDef.Type.LIST, ConfigDef.NO_DEFAULT_VALUE, new NonEmptyListValidator(), ConfigDef.Importance.HIGH, + "Field names on the record value to extract as the record key."); + + private static final String PURPOSE = "copying fields from value to key"; + + private List fields; + + private Cache valueToKeySchemaCache; + + @Override + public void configure(Map configs) { + final SimpleConfig config = new SimpleConfig(CONFIG_DEF, configs); + fields = config.getList(FIELDS_CONFIG); + valueToKeySchemaCache = new SynchronizedCache<>(new LRUCache(16)); + } + + @Override + public R apply(R record) { + if (record.valueSchema() == null) { + return applySchemaless(record); + } else { + return applyWithSchema(record); + } + } + + private R applySchemaless(R record) { + final Map value = requireMap(record.value(), PURPOSE); + final Map key = new HashMap<>(fields.size()); + for (String field : fields) { + key.put(field, value.get(field)); + } + return record.newRecord(record.topic(), record.kafkaPartition(), null, key, record.valueSchema(), record.value(), record.timestamp()); + } + + private R applyWithSchema(R record) { + final Struct value = requireStruct(record.value(), PURPOSE); + + Schema keySchema = valueToKeySchemaCache.get(value.schema()); + if (keySchema == null) { + final SchemaBuilder keySchemaBuilder = SchemaBuilder.struct(); + for (String field : fields) { + final Schema fieldSchema = value.schema().field(field).schema(); + keySchemaBuilder.field(field, fieldSchema); + } + keySchema = keySchemaBuilder.build(); + valueToKeySchemaCache.put(value.schema(), keySchema); + } + + final Struct key = new Struct(keySchema); + for (String field : fields) { + key.put(field, value.get(field)); + } + + return record.newRecord(record.topic(), record.kafkaPartition(), keySchema, key, value.schema(), value, record.timestamp()); + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + + @Override + public void close() { + valueToKeySchemaCache = null; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java new file mode 100644 index 0000000000000..1abbbc8aaf081 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/NonEmptyListValidator.java @@ -0,0 +1,39 @@ +/** + * 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.transforms.util; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.List; + +public class NonEmptyListValidator implements ConfigDef.Validator { + + @Override + public void ensureValid(String name, Object value) { + if (((List) value).isEmpty()) { + throw new ConfigException(name, value, "Empty list"); + } + } + + @Override + public String toString() { + return "non-empty list"; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java new file mode 100644 index 0000000000000..9713b27446f47 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/RegexValidator.java @@ -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. + **/ + +package org.apache.kafka.connect.transforms.util; + +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigException; + +import java.util.regex.Pattern; + +public class RegexValidator implements ConfigDef.Validator { + + @Override + public void ensureValid(String name, Object value) { + try { + Pattern.compile((String) value); + } catch (Exception e) { + throw new ConfigException(name, value, "Invalid regex: " + e.getMessage()); + } + } + + @Override + public String toString() { + return "valid regex"; + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.java new file mode 100644 index 0000000000000..b004f8ae91500 --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/Requirements.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.connect.transforms.util; + +import org.apache.kafka.connect.connector.ConnectRecord; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.sink.SinkRecord; + +import java.util.Map; + +public class Requirements { + + public static void requireSchema(Schema schema, String purpose) { + if (schema == null) { + throw new DataException("Schema required for [" + purpose + "]"); + } + } + + public static Map requireMap(Object value, String purpose) { + if (!(value instanceof Map)) { + throw new DataException("Only Map objects supported in absence of schema for [" + purpose + "], found: " + nullSafeClassName(value)); + } + return (Map) value; + } + + public static Struct requireStruct(Object value, String purpose) { + if (!(value instanceof Struct)) { + throw new DataException("Only Struct objects supported for [" + purpose + "], found: " + nullSafeClassName(value)); + } + return (Struct) value; + } + + public static SinkRecord requireSinkRecord(ConnectRecord record, String purpose) { + if (!(record instanceof SinkRecord)) { + throw new DataException("Only SinkRecord supported for [" + purpose + "], found: " + nullSafeClassName(record)); + } + return (SinkRecord) record; + } + + private static String nullSafeClassName(Object x) { + return x == null ? "null" : x.getClass().getCanonicalName(); + } + +} diff --git a/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java new file mode 100644 index 0000000000000..da261e79b22de --- /dev/null +++ b/connect/transforms/src/main/java/org/apache/kafka/connect/transforms/util/SchemaUtil.java @@ -0,0 +1,40 @@ +/** + * 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.transforms.util; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; + +import java.util.Map; + +public class SchemaUtil { + + public static SchemaBuilder copySchemaBasics(Schema source, SchemaBuilder builder) { + builder.name(source.name()); + builder.version(source.version()); + builder.doc(source.doc()); + + final Map params = source.parameters(); + if (params != null) { + builder.parameters(params); + } + + return builder; + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java new file mode 100644 index 0000000000000..d72179559a50a --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ExtractFieldTest.java @@ -0,0 +1,59 @@ +/** + * 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.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class ExtractFieldTest { + + @Test + public void schemaless() { + final ExtractField xform = new ExtractField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final SinkRecord record = new SinkRecord("test", 0, null, Collections.singletonMap("magic", 42), null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertNull(transformedRecord.keySchema()); + assertEquals(42, transformedRecord.key()); + } + + @Test + public void withSchema() { + final ExtractField xform = new ExtractField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final Schema keySchema = SchemaBuilder.struct().field("magic", Schema.INT32_SCHEMA).build(); + final Struct key = new Struct(keySchema).put("magic", 42); + final SinkRecord record = new SinkRecord("test", 0, keySchema, key, null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertEquals(Schema.INT32_SCHEMA, transformedRecord.keySchema()); + assertEquals(42, transformedRecord.key()); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java similarity index 71% rename from connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java rename to connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java index 99a6e99dde000..b5f9d93d935e4 100644 --- a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistToStructTest.java +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/HoistFieldTest.java @@ -25,12 +25,25 @@ import java.util.Collections; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; -public class HoistToStructTest { +public class HoistFieldTest { @Test - public void sanityCheck() { - final HoistToStruct xform = new HoistToStruct.Key<>(); + public void schemaless() { + final HoistField xform = new HoistField.Key<>(); + xform.configure(Collections.singletonMap("field", "magic")); + + final SinkRecord record = new SinkRecord("test", 0, null, 42, null, null, 0); + final SinkRecord transformedRecord = xform.apply(record); + + assertNull(transformedRecord.keySchema()); + assertEquals(Collections.singletonMap("magic", 42), transformedRecord.key()); + } + + @Test + public void withSchema() { + final HoistField xform = new HoistField.Key<>(); xform.configure(Collections.singletonMap("field", "magic")); final SinkRecord record = new SinkRecord("test", 0, Schema.INT32_SCHEMA, 42, null, null, 0); diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java new file mode 100644 index 0000000000000..c96058afbe6de --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/MaskFieldTest.java @@ -0,0 +1,156 @@ +/** + * 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.transforms; + +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Field; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class MaskFieldTest { + + private static MaskField transform(List fields) { + final MaskField xform = new MaskField.Value<>(); + xform.configure(Collections.singletonMap("fields", fields)); + return xform; + } + + private static SinkRecord record(Schema schema, Object value) { + return new SinkRecord("", 0, null, null, schema, value, 0); + } + + @Test + public void schemaless() { + final Map value = new HashMap<>(); + value.put("magic", 42); + value.put("bool", true); + value.put("byte", (byte) 42); + value.put("short", (short) 42); + value.put("int", 42); + value.put("long", 42L); + value.put("float", 42f); + value.put("double", 42d); + value.put("string", "blabla"); + value.put("date", new Date()); + value.put("bigint", new BigInteger("42")); + value.put("bigdec", new BigDecimal("42.0")); + value.put("list", Collections.singletonList(42)); + value.put("map", Collections.singletonMap("key", "value")); + + final List maskFields = new ArrayList<>(value.keySet()); + maskFields.remove("magic"); + + final Map updatedValue = (Map) transform(maskFields).apply(record(null, value)).value(); + + assertEquals(42, updatedValue.get("magic")); + assertEquals(false, updatedValue.get("bool")); + assertEquals((byte) 0, updatedValue.get("byte")); + assertEquals((short) 0, updatedValue.get("short")); + assertEquals(0, updatedValue.get("int")); + assertEquals(0L, updatedValue.get("long")); + assertEquals(0f, updatedValue.get("float")); + assertEquals(0d, updatedValue.get("double")); + assertEquals("", updatedValue.get("string")); + assertEquals(new Date(0), updatedValue.get("date")); + assertEquals(BigInteger.ZERO, updatedValue.get("bigint")); + assertEquals(BigDecimal.ZERO, updatedValue.get("bigdec")); + assertEquals(Collections.emptyList(), updatedValue.get("list")); + assertEquals(Collections.emptyMap(), updatedValue.get("map")); + } + + @Test + public void withSchema() { + Schema schema = SchemaBuilder.struct() + .field("magic", Schema.INT32_SCHEMA) + .field("bool", Schema.BOOLEAN_SCHEMA) + .field("byte", Schema.INT8_SCHEMA) + .field("short", Schema.INT16_SCHEMA) + .field("int", Schema.INT32_SCHEMA) + .field("long", Schema.INT64_SCHEMA) + .field("float", Schema.FLOAT32_SCHEMA) + .field("double", Schema.FLOAT64_SCHEMA) + .field("string", Schema.STRING_SCHEMA) + .field("date", org.apache.kafka.connect.data.Date.SCHEMA) + .field("time", Time.SCHEMA) + .field("timestamp", Timestamp.SCHEMA) + .field("decimal", Decimal.schema(0)) + .field("array", SchemaBuilder.array(Schema.INT32_SCHEMA)) + .field("map", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA)) + .build(); + + final Struct value = new Struct(schema); + value.put("magic", 42); + value.put("bool", true); + value.put("byte", (byte) 42); + value.put("short", (short) 42); + value.put("int", 42); + value.put("long", 42L); + value.put("float", 42f); + value.put("double", 42d); + value.put("string", "hmm"); + value.put("date", new Date()); + value.put("time", new Date()); + value.put("timestamp", new Date()); + value.put("decimal", new BigDecimal(42)); + value.put("array", Arrays.asList(1, 2, 3)); + value.put("map", Collections.singletonMap("what", "what")); + + final List maskFields = new ArrayList<>(schema.fields().size()); + for (Field field: schema.fields()) { + if (!field.name().equals("magic")) { + maskFields.add(field.name()); + } + } + + final Struct updatedValue = (Struct) transform(maskFields).apply(record(schema, value)).value(); + + assertEquals(42, updatedValue.get("magic")); + assertEquals(false, updatedValue.get("bool")); + assertEquals((byte) 0, updatedValue.get("byte")); + assertEquals((short) 0, updatedValue.get("short")); + assertEquals(0, updatedValue.get("int")); + assertEquals(0L, updatedValue.get("long")); + assertEquals(0f, updatedValue.get("float")); + assertEquals(0d, updatedValue.get("double")); + assertEquals("", updatedValue.get("string")); + assertEquals(new Date(0), updatedValue.get("date")); + assertEquals(new Date(0), updatedValue.get("time")); + assertEquals(new Date(0), updatedValue.get("timestamp")); + assertEquals(BigDecimal.ZERO, updatedValue.get("decimal")); + assertEquals(Collections.emptyList(), updatedValue.get("array")); + assertEquals(Collections.emptyMap(), updatedValue.get("map")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.java new file mode 100644 index 0000000000000..c599265d57200 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/RegexRouterTest.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.connect.transforms; + +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class RegexRouterTest { + + private static String apply(String regex, String replacement, String topic) { + final Map props = new HashMap<>(); + props.put("regex", regex); + props.put("replacement", replacement); + final RegexRouter router = new RegexRouter<>(); + router.configure(props); + return router.apply(new SinkRecord(topic, 0, null, null, null, null, 0)) + .topic(); + } + + @Test + public void staticReplacement() { + assertEquals("bar", apply("foo", "bar", "foo")); + } + + @Test + public void doesntMatch() { + assertEquals("orig", apply("foo", "bar", "orig")); + } + + @Test + public void identity() { + assertEquals("orig", apply("(.*)", "$1", "orig")); + } + + @Test + public void addPrefix() { + assertEquals("prefix-orig", apply("(.*)", "prefix-$1", "orig")); + } + + @Test + public void addSuffix() { + assertEquals("orig-suffix", apply("(.*)", "$1-suffix", "orig")); + } + + @Test + public void slice() { + assertEquals("index", apply("(.*)-(\\d\\d\\d\\d\\d\\d\\d\\d)", "$1", "index-20160117")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java new file mode 100644 index 0000000000000..9f9d4b7031b52 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ReplaceFieldTest.java @@ -0,0 +1,92 @@ +/** + * 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.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class ReplaceFieldTest { + + @Test + public void schemaless() { + final ReplaceField xform = new ReplaceField.Value<>(); + + final Map props = new HashMap<>(); + props.put("blacklist", "dont"); + props.put("renames", "abc:xyz,foo:bar"); + + xform.configure(props); + + final Map value = new HashMap<>(); + value.put("dont", "whatever"); + value.put("abc", 42); + value.put("foo", true); + value.put("etc", "etc"); + + final SinkRecord record = new SinkRecord("test", 0, null, null, null, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Map updatedValue = (Map) transformedRecord.value(); + assertEquals(3, updatedValue.size()); + assertEquals(42, updatedValue.get("xyz")); + assertEquals(true, updatedValue.get("bar")); + assertEquals("etc", updatedValue.get("etc")); + } + + @Test + public void withSchema() { + final ReplaceField xform = new ReplaceField.Value<>(); + + final Map props = new HashMap<>(); + props.put("whitelist", "abc,foo"); + props.put("renames", "abc:xyz,foo:bar"); + + xform.configure(props); + + final Schema schema = SchemaBuilder.struct() + .field("dont", Schema.STRING_SCHEMA) + .field("abc", Schema.INT32_SCHEMA) + .field("foo", Schema.BOOLEAN_SCHEMA) + .field("etc", Schema.STRING_SCHEMA) + .build(); + + final Struct value = new Struct(schema); + value.put("dont", "whatever"); + value.put("abc", 42); + value.put("foo", true); + value.put("etc", "etc"); + + final SinkRecord record = new SinkRecord("test", 0, null, null, schema, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Struct updatedValue = (Struct) transformedRecord.value(); + + assertEquals(2, updatedValue.schema().fields().size()); + assertEquals(new Integer(42), updatedValue.getInt32("xyz")); + assertEquals(true, updatedValue.getBoolean("bar")); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java new file mode 100644 index 0000000000000..2aa790f0a2d82 --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/SetSchemaMetadataTest.java @@ -0,0 +1,67 @@ +/** + * 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.transforms; + +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class SetSchemaMetadataTest { + + @Test + public void schemaNameUpdate() { + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(Collections.singletonMap("schema.name", "foo")); + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + final SinkRecord updatedRecord = xform.apply(record); + assertEquals("foo", updatedRecord.valueSchema().name()); + } + + @Test + public void schemaVersionUpdate() { + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(Collections.singletonMap("schema.version", 42)); + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + final SinkRecord updatedRecord = xform.apply(record); + assertEquals(new Integer(42), updatedRecord.valueSchema().version()); + } + + @Test + public void schemaNameAndVersionUpdate() { + final Map props = new HashMap<>(); + props.put("schema.name", "foo"); + props.put("schema.version", "42"); + + final SetSchemaMetadata xform = new SetSchemaMetadata.Value<>(); + xform.configure(props); + + final SinkRecord record = new SinkRecord("", 0, null, null, SchemaBuilder.struct().build(), null, 0); + + final SinkRecord updatedRecord = xform.apply(record); + + assertEquals("foo", updatedRecord.valueSchema().name()); + assertEquals(new Integer(42), updatedRecord.valueSchema().version()); + } + +} diff --git a/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.java new file mode 100644 index 0000000000000..e5328d36baebb --- /dev/null +++ b/connect/transforms/src/test/java/org/apache/kafka/connect/transforms/ValueToKeyTest.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.connect.transforms; + +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +public class ValueToKeyTest { + + @Test + public void schemaless() { + final ValueToKey xform = new ValueToKey<>(); + xform.configure(Collections.singletonMap("fields", "a,b")); + + final HashMap value = new HashMap<>(); + value.put("a", 1); + value.put("b", 2); + value.put("c", 3); + + final SinkRecord record = new SinkRecord("", 0, null, null, null, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final HashMap expectedKey = new HashMap<>(); + expectedKey.put("a", 1); + expectedKey.put("b", 2); + + assertNull(transformedRecord.keySchema()); + assertEquals(expectedKey, transformedRecord.key()); + } + + @Test + public void withSchema() { + final ValueToKey xform = new ValueToKey<>(); + xform.configure(Collections.singletonMap("fields", "a,b")); + + final Schema valueSchema = SchemaBuilder.struct() + .field("a", Schema.INT32_SCHEMA) + .field("b", Schema.INT32_SCHEMA) + .field("c", Schema.INT32_SCHEMA) + .build(); + + final Struct value = new Struct(valueSchema); + value.put("a", 1); + value.put("b", 2); + value.put("c", 3); + + final SinkRecord record = new SinkRecord("", 0, null, null, valueSchema, value, 0); + final SinkRecord transformedRecord = xform.apply(record); + + final Schema expectedKeySchema = SchemaBuilder.struct() + .field("a", Schema.INT32_SCHEMA) + .field("b", Schema.INT32_SCHEMA) + .build(); + + final Struct expectedKey = new Struct(expectedKeySchema) + .put("a", 1) + .put("b", 2); + + assertEquals(expectedKeySchema, transformedRecord.keySchema()); + assertEquals(expectedKey, transformedRecord.key()); + } + +} diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index 33089d107a9ab..6ccfe941cc953 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -16,6 +16,7 @@ import java.nio.ByteBuffer import java.util.{Collections, Properties} import java.util.concurrent.atomic.AtomicInteger +import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion import kafka.common.KafkaException import kafka.coordinator.GroupOverview import kafka.utils.Logging @@ -28,11 +29,11 @@ import org.apache.kafka.common.network.Selector import org.apache.kafka.common.protocol.{ApiKeys, Errors} import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.OffsetFetchResponse -import org.apache.kafka.common.requests.OffsetFetchResponse.PartitionData import org.apache.kafka.common.utils.{Time, Utils} import org.apache.kafka.common.{Cluster, Node, TopicPartition} import scala.collection.JavaConverters._ +import scala.util.Try class AdminClient(val time: Time, val requestTimeoutMs: Int, @@ -68,37 +69,42 @@ class AdminClient(val time: Time, def findCoordinator(groupId: String): Node = { val requestBuilder = new GroupCoordinatorRequest.Builder(groupId) val response = sendAnyNode(ApiKeys.GROUP_COORDINATOR, requestBuilder).asInstanceOf[GroupCoordinatorResponse] - Errors.forCode(response.errorCode()).maybeThrow() - response.node() + Errors.forCode(response.errorCode).maybeThrow() + response.node } def listGroups(node: Node): List[GroupOverview] = { val response = send(node, ApiKeys.LIST_GROUPS, new ListGroupsRequest.Builder()).asInstanceOf[ListGroupsResponse] - Errors.forCode(response.errorCode()).maybeThrow() - response.groups().asScala.map(group => GroupOverview(group.groupId(), group.protocolType())).toList + Errors.forCode(response.errorCode).maybeThrow() + response.groups.asScala.map(group => GroupOverview(group.groupId, group.protocolType)).toList + } + + def getApiVersions(node: Node): List[ApiVersion] = { + val response = send(node, ApiKeys.API_VERSIONS, new ApiVersionsRequest.Builder()).asInstanceOf[ApiVersionsResponse] + Errors.forCode(response.errorCode).maybeThrow() + response.apiVersions.asScala.toList } private def findAllBrokers(): List[Node] = { val request = MetadataRequest.Builder.allTopics() val response = sendAnyNode(ApiKeys.METADATA, request).asInstanceOf[MetadataResponse] - val errors = response.errors() + val errors = response.errors if (!errors.isEmpty) debug(s"Metadata request contained errors: $errors") - response.cluster().nodes().asScala.toList + response.cluster.nodes.asScala.toList } def listAllGroups(): Map[Node, List[GroupOverview]] = { - findAllBrokers.map { - case broker => - broker -> { - try { - listGroups(broker) - } catch { - case e: Exception => - debug(s"Failed to find groups from broker $broker", e) - List[GroupOverview]() - } + findAllBrokers.map { broker => + broker -> { + try { + listGroups(broker) + } catch { + case e: Exception => + debug(s"Failed to find groups from broker $broker", e) + List[GroupOverview]() } + } }.toMap } @@ -123,9 +129,14 @@ class AdminClient(val time: Time, if (response.hasError) throw response.error.exception response.maybeThrowFirstPartitionError - response.responseData().asScala.map { responseData => (responseData._1, responseData._2.offset) }.toMap + response.responseData.asScala.map { case (tp, partitionData) => (tp, partitionData.offset) }.toMap } + def listAllBrokerVersionInfo(): Map[Node, Try[NodeApiVersions]] = + findAllBrokers.map { broker => + broker -> Try[NodeApiVersions](new NodeApiVersions(getApiVersions(broker).asJava)) + }.toMap + /** * Case class used to represent a consumer of a consumer group */ @@ -220,7 +231,7 @@ object AdminClient { val brokerUrls = config.getList(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) val brokerAddresses = ClientUtils.parseAndValidateAddresses(brokerUrls) val bootstrapCluster = Cluster.bootstrap(brokerAddresses) - metadata.update(bootstrapCluster, 0) + metadata.update(bootstrapCluster, Collections.emptySet(), 0) val selector = new Selector( DefaultConnectionMaxIdleMs, @@ -252,6 +263,6 @@ object AdminClient { time, DefaultRequestTimeoutMs, highLevelClient, - bootstrapCluster.nodes().asScala.toList) + bootstrapCluster.nodes.asScala.toList) } } diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 916db483338de..d4ae4ff946614 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -386,7 +386,7 @@ object AdminUtils extends Logging with AdminUtilities { } def topicExists(zkUtils: ZkUtils, topic: String): Boolean = - zkUtils.zkClient.exists(getTopicPath(topic)) + zkUtils.pathExists(getTopicPath(topic)) def getBrokerMetadatas(zkUtils: ZkUtils, rackAwareMode: RackAwareMode = RackAwareMode.Enforced, brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = { @@ -417,25 +417,26 @@ object AdminUtils extends Logging with AdminUtilities { AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, replicaAssignment, topicConfig) } - def createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils: ZkUtils, - topic: String, - partitionReplicaAssignment: Map[Int, Seq[Int]], - config: Properties = new Properties, - update: Boolean = false, - validateOnly: Boolean = false) { + def validateCreateOrUpdateTopic(zkUtils: ZkUtils, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties, + update: Boolean): Unit = { // validate arguments Topic.validate(topic) - val topicPath = getTopicPath(topic) - if (!update) { - if (zkUtils.zkClient.exists(topicPath)) - throw new TopicExistsException("Topic \"%s\" already exists.".format(topic)) + if (topicExists(zkUtils, topic)) + throw new TopicExistsException(s"Topic '$topic' already exists.") else if (Topic.hasCollisionChars(topic)) { val allTopics = zkUtils.getAllTopics() - val collidingTopics = allTopics.filter(t => Topic.hasCollision(topic, t)) + // check again in case the topic was created in the meantime, otherwise the + // topic could potentially collide with itself + if (allTopics.contains(topic)) + throw new TopicExistsException(s"Topic '$topic' already exists.") + val collidingTopics = allTopics.filter(Topic.hasCollision(topic, _)) if (collidingTopics.nonEmpty) { - throw new InvalidTopicException("Topic \"%s\" collides with existing topics: %s".format(topic, collidingTopics.mkString(", "))) + throw new InvalidTopicException(s"Topic '$topic' collides with existing topics: ${collidingTopics.mkString(", ")}") } } } @@ -450,18 +451,25 @@ object AdminUtils extends Logging with AdminUtilities { // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported - if (!update) { + if (!update) LogConfig.validate(config) - if (!validateOnly) { - // write out the config if there is any, this isn't transactional with the partition assignments - writeEntityConfig(zkUtils, getEntityConfigPath(ConfigType.Topic, topic), config) - } - } + } - if (!validateOnly) { - // create the partition assignment - writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update) + def createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils: ZkUtils, + topic: String, + partitionReplicaAssignment: Map[Int, Seq[Int]], + config: Properties = new Properties, + update: Boolean = false) { + validateCreateOrUpdateTopic(zkUtils, topic, partitionReplicaAssignment, config, update) + + // Configs only matter if a topic is being created. Changing configs via AlterTopic is not supported + if (!update) { + // write out the config if there is any, this isn't transactional with the partition assignments + writeEntityConfig(zkUtils, getEntityConfigPath(ConfigType.Topic, topic), config) } + + // create the partition assignment + writeTopicPartitionAssignment(zkUtils, topic, partitionReplicaAssignment, update) } private def writeTopicPartitionAssignment(zkUtils: ZkUtils, topic: String, replicaAssignment: Map[Int, Seq[Int]], update: Boolean) { @@ -478,7 +486,7 @@ object AdminUtils extends Logging with AdminUtilities { } debug("Updated path %s with %s for replica assignment".format(zkPath, jsonPartitionData)) } catch { - case _: ZkNodeExistsException => throw new TopicExistsException("topic %s already exists".format(topic)) + case _: ZkNodeExistsException => throw new TopicExistsException(s"Topic '$topic' already exists.") case e2: Throwable => throw new AdminOperationException(e2.toString) } } @@ -557,7 +565,7 @@ object AdminUtils extends Logging with AdminUtilities { writeEntityConfig(zkUtils, entityConfigPath, configs) // create the change notification - val seqNode = ZkUtils.EntityConfigChangesPath + "/" + EntityConfigChangeZnodePrefix + val seqNode = ZkUtils.ConfigChangesPath + "/" + EntityConfigChangeZnodePrefix val content = Json.encode(getConfigChangeZnodeData(sanitizedEntityPath)) zkUtils.zkClient.createPersistentSequential(seqNode, content) } diff --git a/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala new file mode 100644 index 0000000000000..812bc9ddad77b --- /dev/null +++ b/core/src/main/scala/kafka/admin/BrokerApiVersionsCommand.scala @@ -0,0 +1,81 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.admin + +import java.io.PrintStream +import java.util.Properties + +import kafka.utils.CommandLineUtils +import org.apache.kafka.common.utils.Utils +import org.apache.kafka.clients.CommonClientConfigs +import joptsimple._ + +import scala.util.{Failure, Success} + +/** + * A command for retrieving broker version information. + */ +object BrokerApiVersionsCommand { + + def main(args: Array[String]): Unit = { + execute(args, System.out) + } + + def execute(args: Array[String], out: PrintStream): Unit = { + val opts = new BrokerVersionCommandOptions(args) + val adminClient = createAdminClient(opts) + val brokerMap = adminClient.listAllBrokerVersionInfo() + brokerMap.foreach { case (broker, versionInfoOrError) => + versionInfoOrError match { + case Success(v) => out.print(s"${broker} -> ${v.toString(true)}\n") + case Failure(v) => out.print(s"${broker} -> ERROR: ${v}\n") + } + } + } + + private def createAdminClient(opts: BrokerVersionCommandOptions): AdminClient = { + val props = if (opts.options.has(opts.commandConfigOpt)) + Utils.loadProps(opts.options.valueOf(opts.commandConfigOpt)) + else + new Properties() + props.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, opts.options.valueOf(opts.bootstrapServerOpt)) + AdminClient.create(props) + } + + class BrokerVersionCommandOptions(args: Array[String]) { + val BootstrapServerDoc = "REQUIRED: The server to connect to." + val CommandConfigDoc = "A property file containing configs to be passed to Admin Client." + + val parser = new OptionParser + val commandConfigOpt = parser.accepts("command-config", CommandConfigDoc) + .withRequiredArg + .describedAs("command config property file") + .ofType(classOf[String]) + val bootstrapServerOpt = parser.accepts("bootstrap-server", BootstrapServerDoc) + .withRequiredArg + .describedAs("server(s) to use for bootstrapping") + .ofType(classOf[String]) + val options = parser.parse(args : _*) + checkArgs() + + def checkArgs() { + // check required args + CommandLineUtils.checkRequiredArgs(parser, options, bootstrapServerOpt) + } + } +} diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index d4e407a6d6c73..c3d9f243ff4ab 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -306,22 +306,23 @@ object ConsumerGroupCommand extends Logging { val offsetFetchResponse = OffsetFetchResponse.readFrom(channel.receive().payload()) offsetFetchResponse.requestInfo.foreach { case (topicAndPartition, offsetAndMetadata) => - if (offsetAndMetadata == OffsetMetadataAndError.NoOffset) { - val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) - // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool - // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) - try { - val offset = zkUtils.readData(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong - offsetMap.put(topicAndPartition, offset) - } catch { - case z: ZkNoNodeException => - printError(s"Could not fetch offset from zookeeper for group '$group' partition '$topicAndPartition' due to missing offset data in zookeeper.", Some(z)) - } + offsetAndMetadata match { + case OffsetMetadataAndError.NoOffset => + val topicDirs = new ZKGroupTopicDirs(group, topicAndPartition.topic) + // this group may not have migrated off zookeeper for offsets storage (we don't expose the dual-commit option in this tool + // (meaning the lag may be off until all the consumers in the group have the same setting for offsets storage) + try { + val offset = zkUtils.readData(topicDirs.consumerOffsetDir + "/" + topicAndPartition.partition)._1.toLong + offsetMap.put(topicAndPartition, offset) + } catch { + case z: ZkNoNodeException => + printError(s"Could not fetch offset from zookeeper for group '$group' partition '$topicAndPartition' due to missing offset data in zookeeper.", Some(z)) + } + case offsetAndMetaData if offsetAndMetaData.error == Errors.NONE.code => + offsetMap.put(topicAndPartition, offsetAndMetadata.offset) + case _ => + printError(s"Could not fetch offset from kafka for group '$group' partition '$topicAndPartition' due to ${Errors.forCode(offsetAndMetadata.error).exception}.") } - else if (offsetAndMetadata.error == Errors.NONE.code) - offsetMap.put(topicAndPartition, offsetAndMetadata.offset) - else - printError(s"Could not fetch offset from kafka for group '$group' partition '$topicAndPartition' due to ${Errors.forCode(offsetAndMetadata.error).exception}.") } channel.disconnect() offsetMap.toMap diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index dc707e56d23d3..4e7b4e0c3937c 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -360,16 +360,17 @@ class ReassignPartitionsCommand(zkUtils: ZkUtils, proposedAssignment: Map[TopicA } private def postRebalanceReplicasThatMoved(existing: Map[TopicAndPartition, Seq[Int]], proposed: Map[TopicAndPartition, Seq[Int]]): Map[TopicAndPartition, Seq[Int]] = { - //For each partition in the proposed list, filter out any replicas that exist now (i.e. are in the proposed list and hence are not moving) - existing.map { case (tp, current) => - tp -> (proposed(tp).toSet -- current).toSeq + //For each partition in the proposed list, filter out any replicas that exist now, and hence aren't being moved. + proposed.map { case (tp, proposedReplicas) => + tp -> (proposedReplicas.toSet -- existing(tp)).toSeq } } private def preRebalanceReplicaForMovingPartitions(existing: Map[TopicAndPartition, Seq[Int]], proposed: Map[TopicAndPartition, Seq[Int]]): Map[TopicAndPartition, Seq[Int]] = { - //Throttle all existing replicas (as any one might be a leader). So just filter out those which aren't moving - existing.filter { case (tp, current) => - (proposed(tp).toSet -- current).nonEmpty + def moving(before: Seq[Int], after: Seq[Int]) = (after.toSet -- before.toSet).nonEmpty + //For any moving partition, throttle all the original (pre move) replicas (as any one might be a leader) + existing.filter { case (tp, preMoveReplicas) => + proposed.contains(tp) && moving(preMoveReplicas, proposed(tp)) } } diff --git a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala index 9bd321c408da4..172233b6c4999 100644 --- a/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala +++ b/core/src/main/scala/kafka/admin/ZkSecurityMigrator.scala @@ -127,7 +127,7 @@ class ZkSecurityMigrator(zkUtils: ZkUtils) extends Logging { private def setAcl(path: String, setPromise: Promise[String]) = { info("Setting ACL for path %s".format(path)) - zkUtils.zkConnection.getZookeeper.setACL(path, ZkUtils.DefaultAcls(zkUtils.isSecure), -1, SetACLCallback, setPromise) + zkUtils.zkConnection.getZookeeper.setACL(path, zkUtils.defaultAcls(path), -1, SetACLCallback, setPromise) } private def getChildren(path: String, childrenPromise: Promise[String]) = { @@ -201,7 +201,7 @@ class ZkSecurityMigrator(zkUtils: ZkUtils) extends Logging { info("Successfully set ACLs for %s".format(path)) promise success "done" case Code.CONNECTIONLOSS => - zkHandle.setACL(path, ZkUtils.DefaultAcls(zkUtils.isSecure), -1, SetACLCallback, ctx) + zkHandle.setACL(path, zkUtils.defaultAcls(path), -1, SetACLCallback, ctx) case Code.NONODE => warn("Znode is gone, it could be have been legitimately deleted: %s".format(path)) promise success "done" @@ -220,12 +220,12 @@ class ZkSecurityMigrator(zkUtils: ZkUtils) extends Logging { private def run(): Unit = { try { setAclIndividually("/") - for (path <- zkUtils.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { debug("Going to set ACL for %s".format(path)) zkUtils.makeSurePersistentPathExists(path) setAclsRecursively(path) } - + @tailrec def recurse(): Unit = { val future = futures.synchronized { diff --git a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala index 2908901cf5aea..dac4cc5d75d4e 100644 --- a/core/src/main/scala/kafka/api/OffsetFetchRequest.scala +++ b/core/src/main/scala/kafka/api/OffsetFetchRequest.scala @@ -25,7 +25,6 @@ import kafka.network.{RequestOrResponseSend, RequestChannel} import kafka.network.RequestChannel.Response import kafka.utils.Logging import org.apache.kafka.common.protocol.{ApiKeys, Errors} -import org.apache.kafka.common.requests.OffsetFetchResponse.PARTITION_ERRORS object OffsetFetchRequest extends Logging { val CurrentVersion: Short = 2 diff --git a/core/src/main/scala/kafka/cluster/Replica.scala b/core/src/main/scala/kafka/cluster/Replica.scala index 346e5d60ea43f..8597b06cdbbce 100644 --- a/core/src/main/scala/kafka/cluster/Replica.scala +++ b/core/src/main/scala/kafka/cluster/Replica.scala @@ -68,9 +68,9 @@ class Replica(val brokerId: Int, */ def updateLogReadResult(logReadResult : LogReadResult) { if (logReadResult.info.fetchOffsetMetadata.messageOffset >= logReadResult.leaderLogEndOffset) - _lastCaughtUpTimeMs = logReadResult.fetchTimeMs + _lastCaughtUpTimeMs = math.max(_lastCaughtUpTimeMs, logReadResult.fetchTimeMs) else if (logReadResult.info.fetchOffsetMetadata.messageOffset >= lastFetchLeaderLogEndOffset) - _lastCaughtUpTimeMs = lastFetchTimeMs + _lastCaughtUpTimeMs = math.max(_lastCaughtUpTimeMs, lastFetchTimeMs) logEndOffset = logReadResult.info.fetchOffsetMetadata lastFetchLeaderLogEndOffset = logReadResult.leaderLogEndOffset @@ -130,6 +130,7 @@ class Replica(val brokerId: Int, replicaString.append("; Topic: " + partition.topic) replicaString.append("; Partition: " + partition.partitionId) replicaString.append("; isLocal: " + isLocal) + replicaString.append("; lastCaughtUpTimeMs: " + lastCaughtUpTimeMs) if (isLocal) replicaString.append("; Highwatermark: " + highWatermark) replicaString.toString } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index d9280348c9422..4bb4b5992172b 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -135,9 +135,13 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf private def removeExistingBroker(brokerState: ControllerBrokerStateInfo) { try { + // Shutdown the RequestSendThread before closing the NetworkClient to avoid the concurrent use of the + // non-threadsafe classes as described in KAFKA-4959. + // The call to shutdownLatch.await() in ShutdownableThread.shutdown() serves as a synchronization barrier that + // hands off the NetworkClient from the RequestSendThread to the ZkEventThread. + brokerState.requestSendThread.shutdown() brokerState.networkClient.close() brokerState.messageQueue.clear() - brokerState.requestSendThread.shutdown() brokerStateInfo.remove(brokerState.brokerNode.id) } catch { case e: Throwable => error("Error while removing broker by the controller", e) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala old mode 100755 new mode 100644 index 8ffa610fe1bf6..774316b9f613f --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -188,7 +188,10 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState if (!isActive) 0 else - controllerContext.partitionLeadershipInfo.count(p => !controllerContext.liveOrShuttingDownBrokerIds.contains(p._2.leaderAndIsr.leader)) + controllerContext.partitionLeadershipInfo.count(p => + (!controllerContext.liveOrShuttingDownBrokerIds.contains(p._2.leaderAndIsr.leader)) + && (!deleteTopicManager.isTopicQueuedUpForDeletion(p._1.topic)) + ) } } } @@ -203,7 +206,10 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState 0 else controllerContext.partitionReplicaAssignment.count { - case (topicPartition, replicas) => controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != replicas.head + case (topicPartition, replicas) => + (controllerContext.partitionLeadershipInfo(topicPartition).leaderAndIsr.leader != replicas.head + && (!deleteTopicManager.isTopicQueuedUpForDeletion(topicPartition.topic)) + ) } } } @@ -1163,9 +1169,16 @@ class KafkaController(val config: KafkaConfig, zkUtils: ZkUtils, val brokerState @throws[Exception] def handleNewSession() { info("ZK expired; shut down all controller components and try to re-elect") - onControllerResignation() - inLock(controllerContext.controllerLock) { - controllerElector.elect + if (controllerElector.getControllerID() != config.brokerId) { + onControllerResignation() + inLock(controllerContext.controllerLock) { + controllerElector.elect + } + } else { + // This can happen when there are multiple consecutive session expiration and handleNewSession() are called multiple + // times. The first call may already register the controller path using the newest ZK session. Therefore, the + // controller path will exist in subsequent calls to handleNewSession(). + info("ZK expired, but the current controller id %d is the same as this broker id, skip re-elect".format(config.brokerId)) } } diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala index 4cbfad6ade6f6..36b0c865f4620 100644 --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala @@ -422,7 +422,7 @@ class GroupCoordinator(val brokerId: Int, } } - def doCommitOffsets(group: GroupMetadata, + private def doCommitOffsets(group: GroupMetadata, memberId: String, generationId: Int, offsetMetadata: immutable.Map[TopicPartition, OffsetAndMetadata], @@ -455,7 +455,7 @@ class GroupCoordinator(val brokerId: Int, } def handleFetchOffsets(groupId: String, - partitions: Option[Seq[TopicPartition]]): (Errors, Map[TopicPartition, OffsetFetchResponse.PartitionData]) = { + partitions: Option[Seq[TopicPartition]] = None): (Errors, Map[TopicPartition, OffsetFetchResponse.PartitionData]) = { if (!isActive.get) (Errors.GROUP_COORDINATOR_NOT_AVAILABLE, Map()) else if (!isCoordinatorForGroup(groupId)) { @@ -497,6 +497,10 @@ class GroupCoordinator(val brokerId: Int, } } + def handleDeletedPartitions(topicPartitions: Seq[TopicPartition]) { + groupManager.cleanupGroupMetadata(Some(topicPartitions)) + } + private def onGroupUnloaded(group: GroupMetadata) { group synchronized { info(s"Unloading group metadata for ${group.groupId} with generation ${group.generationId}") @@ -617,7 +621,7 @@ class GroupCoordinator(val brokerId: Int, val member = new MemberMetadata(memberId, group.groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, protocols) member.awaitingJoinCallback = callback - group.add(member.memberId, member) + group.add(member) maybePrepareRebalance(group) member } diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala index 6b889f474db88..4ea5bdda76261 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadata.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadata.scala @@ -17,8 +17,10 @@ package kafka.coordinator -import collection.mutable +import collection.{Seq, mutable, immutable} + import java.util.UUID + import kafka.common.OffsetAndMetadata import kafka.utils.nonthreadsafe import org.apache.kafka.common.TopicPartition @@ -156,7 +158,7 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou def has(memberId: String) = members.contains(memberId) def get(memberId: String) = members(memberId) - def add(memberId: String, member: MemberMetadata) { + def add(member: MemberMetadata) { if (members.isEmpty) this.protocolType = Some(member.protocolType) @@ -165,8 +167,8 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou assert(supportsProtocols(member.protocols)) if (leaderId == null) - leaderId = memberId - members.put(memberId, member) + leaderId = member.memberId + members.put(member.memberId, member) } def remove(memberId: String) { @@ -263,8 +265,14 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou GroupOverview(groupId, protocolType.getOrElse("")) } + def initializeOffsets(offsets: collection.Map[TopicPartition, OffsetAndMetadata]) { + this.offsets ++= offsets + } + def completePendingOffsetWrite(topicPartition: TopicPartition, offset: OffsetAndMetadata) { - offsets.put(topicPartition, offset) + if (pendingOffsetCommits.contains(topicPartition)) + offsets.put(topicPartition, offset) + pendingOffsetCommits.get(topicPartition) match { case Some(stagedOffset) if offset == stagedOffset => pendingOffsetCommits.remove(topicPartition) case _ => @@ -282,12 +290,20 @@ private[coordinator] class GroupMetadata(val groupId: String, initialState: Grou pendingOffsetCommits ++= offsets } + def removeOffsets(topicPartitions: Seq[TopicPartition]): immutable.Map[TopicPartition, OffsetAndMetadata] = { + topicPartitions.flatMap { topicPartition => + pendingOffsetCommits.remove(topicPartition) + val removedOffset = offsets.remove(topicPartition) + removedOffset.map(topicPartition -> _) + }.toMap + } + def removeExpiredOffsets(startMs: Long) = { val expiredOffsets = offsets.filter { case (topicPartition, offset) => offset.expireTimestamp < startMs && !pendingOffsetCommits.contains(topicPartition) } offsets --= expiredOffsets.keySet - expiredOffsets + expiredOffsets.toMap } def allOffsets = offsets.toMap diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index 74b46adc28f06..ebdd2bb282d47 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -97,7 +97,7 @@ class GroupMetadataManager(val brokerId: Int, unit = TimeUnit.MILLISECONDS) } - def currentGroups(): Iterable[GroupMetadata] = groupMetadataCache.values + def currentGroups: Iterable[GroupMetadata] = groupMetadataCache.values def isPartitionOwned(partition: Int) = inLock(partitionLock) { ownedPartitions.contains(partition) } @@ -133,8 +133,8 @@ class GroupMetadataManager(val brokerId: Int, def prepareStoreGroup(group: GroupMetadata, groupAssignment: Map[String, Array[Byte]], responseCallback: Errors => Unit): Option[DelayedStore] = { - getMagicAndTimestamp(partitionFor(group.groupId)) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(partitionFor(group.groupId)) match { + case Some(magicValue) => val groupMetadataValueVersion = { if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0) 0.toShort @@ -142,6 +142,9 @@ class GroupMetadataManager(val brokerId: Int, GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION } + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() val record = Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion)) @@ -160,16 +163,15 @@ class GroupMetadataManager(val brokerId: Int, // construct the error status in the propagated assignment response // in the cache val status = responseStatus(groupMetadataPartition) - val statusError = Errors.forCode(status.errorCode) - val responseError = if (statusError == Errors.NONE) { + val responseError = if (status.error == Errors.NONE) { Errors.NONE } else { debug(s"Metadata from group ${group.groupId} with generation $generationId failed when appending to log " + - s"due to ${statusError.exceptionName}") + s"due to ${status.error.exceptionName}") // transform the log append error code to the corresponding the commit status error code - statusError match { + status.error match { case Errors.UNKNOWN_TOPIC_OR_PARTITION | Errors.NOT_ENOUGH_REPLICAS | Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND => @@ -186,13 +188,13 @@ class GroupMetadataManager(val brokerId: Int, | Errors.INVALID_FETCH_SIZE => error(s"Appending metadata message for group ${group.groupId} generation $generationId failed due to " + - s"${statusError.exceptionName}, returning UNKNOWN error code to the client") + s"${status.error.exceptionName}, returning UNKNOWN error code to the client") Errors.UNKNOWN case other => error(s"Appending metadata message for group ${group.groupId} generation $generationId failed " + - s"due to unexpected error: ${statusError.exceptionName}") + s"due to unexpected error: ${status.error.exceptionName}") other } @@ -232,11 +234,14 @@ class GroupMetadataManager(val brokerId: Int, } // construct the message set to append - getMagicAndTimestamp(partitionFor(group.groupId)) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(partitionFor(group.groupId)) match { + case Some(magicValue) => + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() val records = filteredOffsetMetadata.map { case (topicPartition, offsetAndMetadata) => Record.create(magicValue, timestampType, timestamp, - GroupMetadataManager.offsetCommitKey(group.groupId, topicPartition.topic, topicPartition.partition), + GroupMetadataManager.offsetCommitKey(group.groupId, topicPartition), GroupMetadataManager.offsetCommitValue(offsetAndMetadata)) }.toSeq @@ -254,11 +259,10 @@ class GroupMetadataManager(val brokerId: Int, // construct the commit response status and insert // the offset and metadata to cache if the append status has no error val status = responseStatus(offsetTopicPartition) - val statusError = Errors.forCode(status.errorCode) val responseCode = group synchronized { - if (statusError == Errors.NONE) { + if (status.error == Errors.NONE) { if (!group.is(Dead)) { filteredOffsetMetadata.foreach { case (topicPartition, offsetAndMetadata) => group.completePendingOffsetWrite(topicPartition, offsetAndMetadata) @@ -273,10 +277,10 @@ class GroupMetadataManager(val brokerId: Int, } debug(s"Offset commit $filteredOffsetMetadata from group ${group.groupId}, consumer $consumerId " + - s"with generation $generationId failed when appending to log due to ${statusError.exceptionName}") + s"with generation $generationId failed when appending to log due to ${status.error.exceptionName}") // transform the log append error code to the corresponding the commit status error code - val responseError = statusError match { + val responseError = status.error match { case Errors.UNKNOWN_TOPIC_OR_PARTITION | Errors.NOT_ENOUGH_REPLICAS | Errors.NOT_ENOUGH_REPLICAS_AFTER_APPEND => @@ -342,19 +346,24 @@ class GroupMetadataManager(val brokerId: Int, (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)) }.toMap } else { - if (topicPartitionsOpt.isEmpty) { - // Return offsets for all partitions owned by this consumer group. (this only applies to consumers that commit offsets to Kafka.) - group.allOffsets.map { case (topicPartition, offsetAndMetadata) => - (topicPartition, new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)) - } - } else { - topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition => - group.offset(topicPartition) match { - case None => (topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE)) - case Some(offsetAndMetadata) => - (topicPartition, new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE)) + topicPartitionsOpt match { + case None => + // Return offsets for all partitions owned by this consumer group. (this only applies to consumers + // that commit offsets to Kafka.) + group.allOffsets.map { case (topicPartition, offsetAndMetadata) => + topicPartition -> new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE) } - }.toMap + + case Some(topicPartitions) => + topicPartitionsOpt.getOrElse(Seq.empty[TopicPartition]).map { topicPartition => + val partitionData = group.offset(topicPartition) match { + case None => + new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NONE) + case Some(offsetAndMetadata) => + new OffsetFetchResponse.PartitionData(offsetAndMetadata.offset, offsetAndMetadata.metadata, Errors.NONE) + } + topicPartition -> partitionData + }.toMap } } } @@ -364,142 +373,149 @@ class GroupMetadataManager(val brokerId: Int, /** * Asynchronously read the partition from the offsets topic and populate the cache */ - def loadGroupsForPartition(offsetsPartition: Int, - onGroupLoaded: GroupMetadata => Unit) { + def loadGroupsForPartition(offsetsPartition: Int, onGroupLoaded: GroupMetadata => Unit) { val topicPartition = new TopicPartition(Topic.GroupMetadataTopicName, offsetsPartition) - scheduler.schedule(topicPartition.toString, loadGroupsAndOffsets) - def loadGroupsAndOffsets() { - info("Loading offsets and group metadata from " + topicPartition) + def doLoadGroupsAndOffsets() { + info(s"Loading offsets and group metadata from $topicPartition") inLock(partitionLock) { if (loadingPartitions.contains(offsetsPartition)) { - info("Offset load from %s already in progress.".format(topicPartition)) + info(s"Offset load from $topicPartition already in progress.") return } else { loadingPartitions.add(offsetsPartition) } } - val startMs = time.milliseconds() try { - replicaManager.logManager.getLog(topicPartition) match { - case Some(log) => - var currOffset = log.logSegments.head.baseOffset - val buffer = ByteBuffer.allocate(config.loadBufferSize) - // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 - val loadedOffsets = mutable.Map[GroupTopicPartition, OffsetAndMetadata]() - val removedOffsets = mutable.Set[GroupTopicPartition]() - val loadedGroups = mutable.Map[String, GroupMetadata]() - val removedGroups = mutable.Set[String]() - - while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) { - buffer.clear() - val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords] - fileRecords.readInto(buffer, 0) - - MemoryRecords.readableRecords(buffer).deepEntries.asScala.foreach { entry => - val record = entry.record - - require(record.hasKey, "Offset entry key should not be null") - val baseKey = GroupMetadataManager.readMessageKey(record.key) - - if (baseKey.isInstanceOf[OffsetKey]) { - // load offset - val key = baseKey.key.asInstanceOf[GroupTopicPartition] - if (record.hasNullValue) { - loadedOffsets.remove(key) - removedOffsets.add(key) - } else { - val value = GroupMetadataManager.readOffsetMessageValue(record.value) - loadedOffsets.put(key, value) - removedOffsets.remove(key) - } + loadGroupsAndOffsets(topicPartition, onGroupLoaded) + } catch { + case t: Throwable => error(s"Error loading offsets from $topicPartition", t) + } finally { + inLock(partitionLock) { + ownedPartitions.add(offsetsPartition) + loadingPartitions.remove(offsetsPartition) + } + } + } + + scheduler.schedule(topicPartition.toString, doLoadGroupsAndOffsets) + } + + private[coordinator] def loadGroupsAndOffsets(topicPartition: TopicPartition, onGroupLoaded: GroupMetadata => Unit) { + def highWaterMark = replicaManager.getHighWatermark(topicPartition).getOrElse(-1L) + + val startMs = time.milliseconds() + replicaManager.getLog(topicPartition) match { + case None => + warn(s"Attempted to load offsets and group metadata from $topicPartition, but found no log") + + case Some(log) => + var currOffset = log.logStartOffset + val buffer = ByteBuffer.allocate(config.loadBufferSize) + // loop breaks if leader changes at any time during the load, since getHighWatermark is -1 + val loadedOffsets = mutable.Map[GroupTopicPartition, OffsetAndMetadata]() + val removedOffsets = mutable.Set[GroupTopicPartition]() + val loadedGroups = mutable.Map[String, GroupMetadata]() + val removedGroups = mutable.Set[String]() + + while (currOffset < highWaterMark && !shuttingDown.get()) { + buffer.clear() + val fileRecords = log.read(currOffset, config.loadBufferSize, maxOffset = None, minOneMessage = true) + .records.asInstanceOf[FileRecords] + val bufferRead = fileRecords.readInto(buffer, 0) + + MemoryRecords.readableRecords(bufferRead).deepEntries.asScala.foreach { entry => + val record = entry.record + require(record.hasKey, "Group metadata/offset entry key should not be null") + + GroupMetadataManager.readMessageKey(record.key) match { + case offsetKey: OffsetKey => + // load offset + val key = offsetKey.key + if (record.hasNullValue) { + loadedOffsets.remove(key) + removedOffsets.add(key) } else { - // load group metadata - val groupId = baseKey.key.asInstanceOf[String] - val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value) - if (groupMetadata != null) { - trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}") - removedGroups.remove(groupId) - loadedGroups.put(groupId, groupMetadata) - } else { - loadedGroups.remove(groupId) - removedGroups.add(groupId) - } + val value = GroupMetadataManager.readOffsetMessageValue(record.value) + loadedOffsets.put(key, value) + removedOffsets.remove(key) } - currOffset = entry.nextOffset - } - } - - val (groupOffsets, noGroupOffsets) = loadedOffsets - .groupBy(_._1.group) - .mapValues(_.map{ case (groupTopicPartition, offsetAndMetadata) => (groupTopicPartition.topicPartition, offsetAndMetadata)}) - .partition(value => loadedGroups.contains(value._1)) + case groupMetadataKey: GroupMetadataKey => + // load group metadata + val groupId = groupMetadataKey.key + val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value) + if (groupMetadata != null) { + trace(s"Loaded group metadata for group $groupId with generation ${groupMetadata.generationId}") + removedGroups.remove(groupId) + loadedGroups.put(groupId, groupMetadata) + } else { + loadedGroups.remove(groupId) + removedGroups.add(groupId) + } - loadedGroups.values.foreach { group => - val offsets = groupOffsets.getOrElse(group.groupId, Map.empty) - loadGroup(group, offsets) - onGroupLoaded(group) + case unknownKey => + throw new IllegalStateException(s"Unexpected message key $unknownKey while loading offsets and group metadata") } - noGroupOffsets.foreach { case (groupId, offsets) => - val group = new GroupMetadata(groupId) - loadGroup(group, offsets) - onGroupLoaded(group) - } + currOffset = entry.nextOffset + } + } - removedGroups.foreach { groupId => - if (groupMetadataCache.contains(groupId)) - throw new IllegalStateException(s"Unexpected unload of active group $groupId while " + - s"loading partition $topicPartition") - } + val (groupOffsets, emptyGroupOffsets) = loadedOffsets + .groupBy(_._1.group) + .mapValues(_.map { case (groupTopicPartition, offset) => (groupTopicPartition.topicPartition, offset)} ) + .partition { case (group, _) => loadedGroups.contains(group) } - if (!shuttingDown.get()) - info("Finished loading offsets from %s in %d milliseconds." - .format(topicPartition, time.milliseconds() - startMs)) - case None => - warn("No log found for " + topicPartition) + loadedGroups.values.foreach { group => + val offsets = groupOffsets.getOrElse(group.groupId, Map.empty[TopicPartition, OffsetAndMetadata]) + loadGroup(group, offsets) + onGroupLoaded(group) } - } - catch { - case t: Throwable => - error("Error in loading offsets from " + topicPartition, t) - } - finally { - inLock(partitionLock) { - ownedPartitions.add(offsetsPartition) - loadingPartitions.remove(offsetsPartition) + + // load groups which store offsets in kafka, but which have no active members and thus no group + // metadata stored in the log + emptyGroupOffsets.foreach { case (groupId, offsets) => + val group = new GroupMetadata(groupId) + loadGroup(group, offsets) + onGroupLoaded(group) } - } + + removedGroups.foreach { groupId => + // if the cache already contains a group which should be removed, raise an error. Note that it + // is possible (however unlikely) for a consumer group to be removed, and then to be used only for + // offset storage (i.e. by "simple" consumers) + if (groupMetadataCache.contains(groupId) && !emptyGroupOffsets.contains(groupId)) + throw new IllegalStateException(s"Unexpected unload of active group $groupId while " + + s"loading partition $topicPartition") + } + + if (!shuttingDown.get()) + info("Finished loading offsets from %s in %d milliseconds." + .format(topicPartition, time.milliseconds() - startMs)) } } - private def loadGroup(group: GroupMetadata, offsets: Iterable[(TopicPartition, OffsetAndMetadata)]): Unit = { + private def loadGroup(group: GroupMetadata, offsets: Map[TopicPartition, OffsetAndMetadata]): Unit = { + // offsets are initialized prior to loading the group into the cache to ensure that clients see a consistent + // view of the group's offsets + val loadedOffsets = offsets.mapValues { offsetAndMetadata => + // special handling for version 0: + // set the expiration time stamp as commit time stamp + server default retention time + if (offsetAndMetadata.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) + offsetAndMetadata.copy(expireTimestamp = offsetAndMetadata.commitTimestamp + config.offsetsRetentionMs) + else + offsetAndMetadata + } + trace(s"Initialized offsets $loadedOffsets for group ${group.groupId}") + group.initializeOffsets(loadedOffsets) + val currentGroup = addGroup(group) - if (group != currentGroup) { + if (group != currentGroup) debug(s"Attempt to load group ${group.groupId} from log with generation ${group.generationId} failed " + s"because there is already a cached group with generation ${currentGroup.generationId}") - } else { - - offsets.foreach { - case (topicPartition, offsetAndMetadata) => { - val offset = offsetAndMetadata.copy ( - expireTimestamp = { - // special handling for version 0: - // set the expiration time stamp as commit time stamp + server default retention time - if (offsetAndMetadata.expireTimestamp == org.apache.kafka.common.requests.OffsetCommitRequest.DEFAULT_TIMESTAMP) - offsetAndMetadata.commitTimestamp + config.offsetsRetentionMs - else - offsetAndMetadata.expireTimestamp - } - ) - trace("Loaded offset %s for %s.".format(offset, topicPartition)) - group.completePendingOffsetWrite(topicPartition, offset) - } - } - } } /** @@ -541,33 +557,44 @@ class GroupMetadataManager(val brokerId: Int, } // visible for testing - private[coordinator] def cleanupGroupMetadata() { + private[coordinator] def cleanupGroupMetadata(): Unit = { + cleanupGroupMetadata(None) + } + + def cleanupGroupMetadata(deletedTopicPartitions: Option[Seq[TopicPartition]]) { val startMs = time.milliseconds() var offsetsRemoved = 0 groupMetadataCache.foreach { case (groupId, group) => - val (expiredOffsets, groupIsDead, generation) = group synchronized { - // remove expired offsets from the cache - val expiredOffsets = group.removeExpiredOffsets(startMs) + val (removedOffsets, groupIsDead, generation) = group synchronized { + val removedOffsets = deletedTopicPartitions match { + case Some(topicPartitions) => group.removeOffsets(topicPartitions) + case None => group.removeExpiredOffsets(startMs) + } + if (group.is(Empty) && !group.hasOffsets) { info(s"Group $groupId transitioned to Dead in generation ${group.generationId}") group.transitionTo(Dead) } - (expiredOffsets, group.is(Dead), group.generationId) + (removedOffsets, group.is(Dead), group.generationId) } val offsetsPartition = partitionFor(groupId) val appendPartition = new TopicPartition(Topic.GroupMetadataTopicName, offsetsPartition) - getMagicAndTimestamp(offsetsPartition) match { - case Some((magicValue, timestampType, timestamp)) => + getMagic(offsetsPartition) match { + case Some(magicValue) => + // We always use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + val timestampType = TimestampType.CREATE_TIME + val timestamp = time.milliseconds() + val partitionOpt = replicaManager.getPartition(appendPartition) partitionOpt.foreach { partition => - val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) => - trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata") - val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition) + val tombstones = removedOffsets.map { case (topicPartition, offsetAndMetadata) => + trace(s"Removing expired/deleted offset and metadata for $groupId, $topicPartition: $offsetAndMetadata") + val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition) Record.create(magicValue, timestampType, timestamp, commitKey, null) }.toBuffer - trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.") + trace(s"Marked ${removedOffsets.size} offsets in $appendPartition for deletion.") // We avoid writing the tombstone when the generationId is 0, since this group is only using // Kafka for offset storage. @@ -584,11 +611,11 @@ class GroupMetadataManager(val brokerId: Int, // do not need to require acks since even if the tombstone is lost, // it will be appended again in the next purge cycle partition.appendRecordsToLeader(MemoryRecords.withRecords(timestampType, compressionType, tombstones: _*)) - offsetsRemoved += expiredOffsets.size - trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId") + offsetsRemoved += removedOffsets.size + trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired/deleted offsets and/or metadata for group $groupId") } catch { case t: Throwable => - error(s"Failed to append ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId.", t) + error(s"Failed to append ${tombstones.size} tombstones to $appendPartition for expired/deleted offsets and/or metadata for group $groupId.", t) // ignore and continue } } @@ -602,16 +629,6 @@ class GroupMetadataManager(val brokerId: Int, info(s"Removed $offsetsRemoved expired offsets in ${time.milliseconds() - startMs} milliseconds.") } - private def getHighWatermark(partitionId: Int): Long = { - val partitionOpt = replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, partitionId)) - - val hw = partitionOpt.map { partition => - partition.leaderReplicaIfLocal.map(_.highWatermark.messageOffset).getOrElse(-1L) - }.getOrElse(-1L) - - hw - } - /* * Check if the offset metadata length is valid */ @@ -645,15 +662,10 @@ class GroupMetadataManager(val brokerId: Int, * Check if the replica is local and return the message format version and timestamp * * @param partition Partition of GroupMetadataTopic - * @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise + * @return Some(MessageFormatVersion) if replica is local, None otherwise */ - private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = { - val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partition) - replicaManager.getMagicAndTimestampType(groupMetadataTopicPartition).map { case (messageFormatVersion, timestampType) => - val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds() - (messageFormatVersion, timestampType, timestamp) - } - } + private def getMagic(partition: Int): Option[Byte] = + replicaManager.getMagic(new TopicPartition(Topic.GroupMetadataTopicName, partition)) /** * Add the partition into the owned list @@ -812,11 +824,12 @@ object GroupMetadataManager { * * @return key for offset commit message */ - private def offsetCommitKey(group: String, topic: String, partition: Int, versionId: Short = 0): Array[Byte] = { + private[coordinator] def offsetCommitKey(group: String, topicPartition: TopicPartition, + versionId: Short = 0): Array[Byte] = { val key = new Struct(CURRENT_OFFSET_KEY_SCHEMA) key.set(OFFSET_KEY_GROUP_FIELD, group) - key.set(OFFSET_KEY_TOPIC_FIELD, topic) - key.set(OFFSET_KEY_PARTITION_FIELD, partition) + key.set(OFFSET_KEY_TOPIC_FIELD, topicPartition.topic) + key.set(OFFSET_KEY_PARTITION_FIELD, topicPartition.partition) val byteBuffer = ByteBuffer.allocate(2 /* version */ + key.sizeOf) byteBuffer.putShort(CURRENT_OFFSET_KEY_SCHEMA_VERSION) @@ -829,7 +842,7 @@ object GroupMetadataManager { * * @return key bytes for group metadata message */ - def groupMetadataKey(group: String): Array[Byte] = { + private[coordinator] def groupMetadataKey(group: String): Array[Byte] = { val key = new Struct(CURRENT_GROUP_KEY_SCHEMA) key.set(GROUP_KEY_GROUP_FIELD, group) @@ -845,7 +858,7 @@ object GroupMetadataManager { * @param offsetAndMetadata consumer's current offset and metadata * @return payload for offset commit message */ - private def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { + private[coordinator] def offsetCommitValue(offsetAndMetadata: OffsetAndMetadata): Array[Byte] = { // generate commit value with schema version 1 val value = new Struct(CURRENT_OFFSET_VALUE_SCHEMA) value.set(OFFSET_VALUE_OFFSET_FIELD_V1, offsetAndMetadata.offset) @@ -867,9 +880,9 @@ object GroupMetadataManager { * @param version the version of the value message to use * @return payload for offset commit message */ - def groupMetadataValue(groupMetadata: GroupMetadata, - assignment: Map[String, Array[Byte]], - version: Short = 0): Array[Byte] = { + private[coordinator] def groupMetadataValue(groupMetadata: GroupMetadata, + assignment: Map[String, Array[Byte]], + version: Short = 0): Array[Byte] = { val value = if (version == 0) new Struct(GROUP_METADATA_VALUE_SCHEMA_V0) else new Struct(CURRENT_GROUP_VALUE_SCHEMA) value.set(PROTOCOL_TYPE_KEY, groupMetadata.protocolType.getOrElse("")) @@ -877,26 +890,25 @@ object GroupMetadataManager { value.set(PROTOCOL_KEY, groupMetadata.protocol) value.set(LEADER_KEY, groupMetadata.leaderId) - val memberArray = groupMetadata.allMemberMetadata.map { - case memberMetadata => - val memberStruct = value.instance(MEMBERS_KEY) - memberStruct.set(MEMBER_ID_KEY, memberMetadata.memberId) - memberStruct.set(CLIENT_ID_KEY, memberMetadata.clientId) - memberStruct.set(CLIENT_HOST_KEY, memberMetadata.clientHost) - memberStruct.set(SESSION_TIMEOUT_KEY, memberMetadata.sessionTimeoutMs) + val memberArray = groupMetadata.allMemberMetadata.map { memberMetadata => + val memberStruct = value.instance(MEMBERS_KEY) + memberStruct.set(MEMBER_ID_KEY, memberMetadata.memberId) + memberStruct.set(CLIENT_ID_KEY, memberMetadata.clientId) + memberStruct.set(CLIENT_HOST_KEY, memberMetadata.clientHost) + memberStruct.set(SESSION_TIMEOUT_KEY, memberMetadata.sessionTimeoutMs) - if (version > 0) - memberStruct.set(REBALANCE_TIMEOUT_KEY, memberMetadata.rebalanceTimeoutMs) + if (version > 0) + memberStruct.set(REBALANCE_TIMEOUT_KEY, memberMetadata.rebalanceTimeoutMs) - val metadata = memberMetadata.metadata(groupMetadata.protocol) - memberStruct.set(SUBSCRIPTION_KEY, ByteBuffer.wrap(metadata)) + val metadata = memberMetadata.metadata(groupMetadata.protocol) + memberStruct.set(SUBSCRIPTION_KEY, ByteBuffer.wrap(metadata)) - val memberAssignment = assignment(memberMetadata.memberId) - assert(memberAssignment != null) + val memberAssignment = assignment(memberMetadata.memberId) + assert(memberAssignment != null) - memberStruct.set(ASSIGNMENT_KEY, ByteBuffer.wrap(memberAssignment)) + memberStruct.set(ASSIGNMENT_KEY, ByteBuffer.wrap(memberAssignment)) - memberStruct + memberStruct } value.set(MEMBERS_KEY, memberArray.toArray) @@ -1010,7 +1022,7 @@ object GroupMetadataManager { member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer]) - group.add(memberId, member) + group.add(member) } group diff --git a/core/src/main/scala/kafka/log/AbstractIndex.scala b/core/src/main/scala/kafka/log/AbstractIndex.scala index 77ef0f73f050e..6f850d54ecf75 100644 --- a/core/src/main/scala/kafka/log/AbstractIndex.scala +++ b/core/src/main/scala/kafka/log/AbstractIndex.scala @@ -17,7 +17,7 @@ package kafka.log -import java.io.{File, RandomAccessFile} +import java.io.{File, IOException, RandomAccessFile} import java.nio.{ByteBuffer, MappedByteBuffer} import java.nio.channels.FileChannel import java.util.concurrent.locks.{Lock, ReentrantLock} @@ -141,8 +141,16 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon */ def delete(): Boolean = { info(s"Deleting index ${file.getAbsolutePath}") - if(Os.isWindows) + inLock(lock) { + // On JVM, a memory mapping is typically unmapped by garbage collector. + // However, in some cases it can pause application threads(STW) for a long moment reading metadata from a physical disk. + // To prevent this, we forcefully cleanup memory mapping within proper execution which never affects API responsiveness. + // See https://issues.apache.org/jira/browse/KAFKA-4614 for the details. CoreUtils.swallow(forceUnmap(mmap)) + // Accessing unmapped mmap crashes JVM by SEGV. + // Accessing it after this method called sounds like a bug but for safety, assign null and do not allow later access. + mmap = null + } file.delete() } @@ -185,7 +193,7 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon def truncateTo(offset: Long): Unit /** - * Forcefully free the buffer's mmap. We do this only on windows. + * Forcefully free the buffer's mmap. */ protected def forceUnmap(m: MappedByteBuffer) { try { @@ -279,4 +287,4 @@ abstract class AbstractIndex[K, V](@volatile var file: File, val baseOffset: Lon object IndexSearchType extends Enumeration { type IndexSearchEntity = Value val KEY, VALUE = Value -} \ No newline at end of file +} diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 8dea5ca25c08e..417122c0d7977 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -206,6 +206,7 @@ class Log(@volatile var dir: File, val timeIndexFile = Log.timeIndexFilename(dir, start) val indexFileExists = indexFile.exists() + val timeIndexFileExists = timeIndexFile.exists() val segment = new LogSegment(dir = dir, startOffset = start, indexIntervalBytes = config.indexInterval, @@ -217,6 +218,9 @@ class Log(@volatile var dir: File, if (indexFileExists) { try { segment.index.sanityCheck() + // Resize the time index file to 0 if it is newly created. + if (!timeIndexFileExists) + segment.timeIndex.resize(0) segment.timeIndex.sanityCheck() } catch { case e: java.lang.IllegalArgumentException => @@ -316,7 +320,7 @@ class Log(@volatile var dir: File, /** * Check if we have the "clean shutdown" file */ - private def hasCleanShutdownFile() = new File(dir.getParentFile, CleanShutdownFile).exists() + private def hasCleanShutdownFile = new File(dir.getParentFile, CleanShutdownFile).exists() /** * The number of segments in the log. diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index eaa60160c92e6..55669c061ec2c 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -323,23 +323,12 @@ object LogConfig { throw new InvalidConfigurationException(s"Unknown Log configuration $name.") } - /** - * Check that the property values are valid relative to each other - */ - def validateValues(props: Properties) { - val segmentBytes = if (props.getProperty(SegmentBytesProp) == null) Defaults.SegmentSize else props.getProperty(SegmentBytesProp).toLong - val retentionBytes = if (props.getProperty(RetentionBytesProp) == null) Defaults.RetentionSize else props.getProperty(RetentionBytesProp).toLong - if (segmentBytes > retentionBytes && retentionBytes != -1) - throw new InvalidConfigurationException(s"segment.bytes ${segmentBytes} is not less than or equal to retention.bytes ${retentionBytes}") - } - /** * Check that the given properties contain only valid log config names and that all values can be parsed and are valid */ def validate(props: Properties) { validateNames(props) configDef.parse(props) - validateValues(props) } } diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index 8cd9b34f6e5f8..761edf9ac762c 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -349,13 +349,7 @@ class LogManager(val logDirs: Array[File], /** * Get the log if it exists, otherwise return None */ - def getLog(topicPartition: TopicPartition): Option[Log] = { - val log = logs.get(topicPartition) - if (log == null) - None - else - Some(log) - } + def getLog(topicPartition: TopicPartition): Option[Log] = Option(logs.get(topicPartition)) /** * Create a log for the given topic and the given partition @@ -363,28 +357,20 @@ class LogManager(val logDirs: Array[File], */ def createLog(topicPartition: TopicPartition, config: LogConfig): Log = { logCreationOrDeletionLock synchronized { - var log = logs.get(topicPartition) - - // check if the log has already been created in another thread - if(log != null) - return log - - // if not, create it - val dataDir = nextLogDir() - val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition) - dir.mkdirs() - log = new Log(dir, - config, - recoveryPoint = 0L, - scheduler, - time) - logs.put(topicPartition, log) - info("Created log for partition [%s,%d] in %s with properties {%s}." - .format(topicPartition.topic, - topicPartition.partition, - dataDir.getAbsolutePath, - config.originals.asScala.mkString(", "))) - log + // create the log if it has not already been created in another thread + getLog(topicPartition).getOrElse { + val dataDir = nextLogDir() + val dir = new File(dataDir, topicPartition.topic + "-" + topicPartition.partition) + dir.mkdirs() + val log = new Log(dir, config, recoveryPoint = 0L, scheduler, time) + logs.put(topicPartition, log) + info("Created log for partition [%s,%d] in %s with properties {%s}." + .format(topicPartition.topic, + topicPartition.partition, + dataDir.getAbsolutePath, + config.originals.asScala.mkString(", "))) + log + } } } diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index a0e7294a19afa..28f910a199e5a 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -56,10 +56,10 @@ object SimpleAclAuthorizer { * /kafka-acl/ConsumerGroup/group-1 => {"version": 1, "acls": [ { "host":"host1", "permissionType": "Allow","operation": "Read","principal": "User:alice"}]} * */ - val AclZkPath = "/kafka-acl" + val AclZkPath = ZkUtils.KafkaAclPath //notification node which gets updated with the resource name when acl on a resource is changed. - val AclChangedZkPath = "/kafka-acl-changes" + val AclChangedZkPath = ZkUtils.KafkaAclChangesPath //prefix of all the change notification sequence node. val AclChangedPrefix = "acl_changes_" diff --git a/core/src/main/scala/kafka/server/AdminManager.scala b/core/src/main/scala/kafka/server/AdminManager.scala index 75a71f5124083..98a4b1897d635 100644 --- a/core/src/main/scala/kafka/server/AdminManager.scala +++ b/core/src/main/scala/kafka/server/AdminManager.scala @@ -23,7 +23,7 @@ import kafka.common.TopicAlreadyMarkedForDeletionException import kafka.log.LogConfig import kafka.metrics.KafkaMetricsGroup import kafka.utils._ -import org.apache.kafka.common.errors.InvalidRequestException +import org.apache.kafka.common.errors.{ApiException, InvalidRequestException, PolicyViolationException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.CreateTopicsRequest._ @@ -43,7 +43,7 @@ class AdminManager(val config: KafkaConfig, private val topicPurgatory = DelayedOperationPurgatory[DelayedOperation]("topic", config.brokerId) private val createTopicPolicy = - Option(config.getConfiguredInstance(KafkaConfig.CreateTopicsPolicyClassNameProp, classOf[CreateTopicPolicy])) + Option(config.getConfiguredInstance(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[CreateTopicPolicy])) def hasDelayedTopicOperations = topicPurgatory.delayed() != 0 @@ -80,29 +80,49 @@ class AdminManager(val config: KafkaConfig, && !arguments.replicasAssignments.isEmpty) throw new InvalidRequestException("Both numPartitions or replicationFactor and replicasAssignments were set. " + "Both cannot be used at the same time.") - else { - createTopicPolicy.foreach(_.validate(new RequestMetadata(topic, arguments.numPartitions, - arguments.replicationFactor, arguments.replicasAssignments, arguments.configs))) - - if (!arguments.replicasAssignments.isEmpty) { - // Note: we don't check that replicaAssignment doesn't contain unknown brokers - unlike in add-partitions case, - // this follows the existing logic in TopicCommand - arguments.replicasAssignments.asScala.map { case (partitionId, replicas) => - (partitionId.intValue, replicas.asScala.map(_.intValue)) - } - } else { - AdminUtils.assignReplicasToBrokers(brokers, arguments.numPartitions, arguments.replicationFactor) + else if (!arguments.replicasAssignments.isEmpty) { + // Note: we don't check that replicaAssignment contains unknown brokers - unlike in add-partitions case, + // this follows the existing logic in TopicCommand + arguments.replicasAssignments.asScala.map { case (partitionId, replicas) => + (partitionId.intValue, replicas.asScala.map(_.intValue)) } - } + } else + AdminUtils.assignReplicasToBrokers(brokers, arguments.numPartitions, arguments.replicationFactor) } trace(s"Assignments for topic $topic are $assignments ") - AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, - update = false, validateOnly = validateOnly) + + createTopicPolicy match { + case Some(policy) => + AdminUtils.validateCreateOrUpdateTopic(zkUtils, topic, assignments, configs, update = false) + + // Use `null` for unset fields in the public API + val numPartitions: java.lang.Integer = + if (arguments.numPartitions == NO_NUM_PARTITIONS) null else arguments.numPartitions + val replicationFactor: java.lang.Short = + if (arguments.replicationFactor == NO_REPLICATION_FACTOR) null else arguments.replicationFactor + val replicaAssignments = if (arguments.replicasAssignments.isEmpty) null else arguments.replicasAssignments + + policy.validate(new RequestMetadata(topic, numPartitions, replicationFactor, replicaAssignments, + arguments.configs)) + + if (!validateOnly) + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, update = false) + + case None => + if (validateOnly) + AdminUtils.validateCreateOrUpdateTopic(zkUtils, topic, assignments, configs, update = false) + else + AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, assignments, configs, update = false) + } CreateTopicMetadata(topic, assignments, new CreateTopicsResponse.Error(Errors.NONE, null)) } catch { - case e: Throwable => + // Log client errors at a lower level than unexpected exceptions + case e@ (_: PolicyViolationException | _: ApiException) => info(s"Error processing create topic request for topic $topic with arguments $arguments", e) CreateTopicMetadata(topic, Map(), new CreateTopicsResponse.Error(Errors.forException(e), e.getMessage)) + case e: Throwable => + error(s"Error processing create topic request for topic $topic with arguments $arguments", e) + CreateTopicMetadata(topic, Map(), new CreateTopicsResponse.Error(Errors.forException(e), e.getMessage)) } } @@ -171,5 +191,6 @@ class AdminManager(val config: KafkaConfig, def shutdown() { topicPurgatory.shutdown() + CoreUtils.swallow(createTopicPolicy.foreach(_.close())) } } diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala index 0a1884a94a3fa..a05131ae8c6ab 100644 --- a/core/src/main/scala/kafka/server/DelayedFetch.scala +++ b/core/src/main/scala/kafka/server/DelayedFetch.scala @@ -150,7 +150,7 @@ class DelayedFetch(delayMs: Long, ) val fetchPartitionData = logReadResults.map { case (tp, result) => - tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records) + tp -> FetchPartitionData(result.error, result.hw, result.info.records) } responseCallback(fetchPartitionData) diff --git a/core/src/main/scala/kafka/server/DelayedProduce.scala b/core/src/main/scala/kafka/server/DelayedProduce.scala index 1af0bfbfa9c9b..f27dff3142f0b 100644 --- a/core/src/main/scala/kafka/server/DelayedProduce.scala +++ b/core/src/main/scala/kafka/server/DelayedProduce.scala @@ -33,7 +33,7 @@ case class ProducePartitionStatus(requiredOffset: Long, responseStatus: Partitio @volatile var acksPending = false override def toString = "[acksPending: %b, error: %d, startOffset: %d, requiredOffset: %d]" - .format(acksPending, responseStatus.errorCode, responseStatus.baseOffset, requiredOffset) + .format(acksPending, responseStatus.error.code, responseStatus.baseOffset, requiredOffset) } /** @@ -58,10 +58,10 @@ class DelayedProduce(delayMs: Long, // first update the acks pending variable according to the error code produceMetadata.produceStatus.foreach { case (topicPartition, status) => - if (status.responseStatus.errorCode == Errors.NONE.code) { + if (status.responseStatus.error == Errors.NONE) { // Timeout error state will be cleared when required acks are received status.acksPending = true - status.responseStatus.errorCode = Errors.REQUEST_TIMED_OUT.code + status.responseStatus.error = Errors.REQUEST_TIMED_OUT } else { status.acksPending = false } @@ -95,7 +95,7 @@ class DelayedProduce(delayMs: Long, // Case B.1 || B.2 if (error != Errors.NONE || hasEnough) { status.acksPending = false - status.responseStatus.errorCode = error.code + status.responseStatus.error = error } } } diff --git a/core/src/main/scala/kafka/server/DynamicConfigManager.scala b/core/src/main/scala/kafka/server/DynamicConfigManager.scala index e0e6a03cec410..c81ce6c5e3551 100644 --- a/core/src/main/scala/kafka/server/DynamicConfigManager.scala +++ b/core/src/main/scala/kafka/server/DynamicConfigManager.scala @@ -148,7 +148,7 @@ class DynamicConfigManager(private val zkUtils: ZkUtils, } } - private val configChangeListener = new ZkNodeChangeNotificationListener(zkUtils, ZkUtils.EntityConfigChangesPath, AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) + private val configChangeListener = new ZkNodeChangeNotificationListener(zkUtils, ZkUtils.ConfigChangesPath, AdminUtils.EntityConfigChangeZnodePrefix, ConfigChangedNotificationHandler) /** * Begin watching for config changes diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index a6ad7b2ec8799..68c3dce1804db 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -38,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils} import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException} import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.network.ListenerName -import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol} import org.apache.kafka.common.record.{MemoryRecords, Record} import org.apache.kafka.common.requests._ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse @@ -196,7 +196,10 @@ class KafkaApis(val requestChannel: RequestChannel, val updateMetadataResponse = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { - replicaManager.maybeUpdateMetadataCache(correlationId, updateMetadataRequest, metadataCache) + val deletedPartitions = replicaManager.maybeUpdateMetadataCache(correlationId, updateMetadataRequest, metadataCache) + if (deletedPartitions.nonEmpty) + coordinator.handleDeletedPartitions(deletedPartitions) + if (adminManager.hasDelayedTopicOperations) { updateMetadataRequest.partitionStates.keySet.asScala.map(_.topic).foreach { topic => adminManager.tryCompleteDelayedTopicOperations(topic) @@ -346,33 +349,32 @@ class KafkaApis(val requestChannel: RequestChannel, val produceRequest = request.body.asInstanceOf[ProduceRequest] val numBytesAppended = request.header.sizeOf + produceRequest.sizeOf - val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = produceRequest.partitionRecords.asScala.partition { - case (topicPartition, _) => authorize(request.session, Describe, new Resource(auth.Topic, topicPartition.topic)) && metadataCache.contains(topicPartition.topic) - } + val (existingAndAuthorizedForDescribeTopics, nonExistingOrUnauthorizedForDescribeTopics) = + produceRequest.partitionRecordsOrFail.asScala.partition { case (tp, _) => + authorize(request.session, Describe, new Resource(auth.Topic, tp.topic)) && metadataCache.contains(tp.topic) + } val (authorizedRequestInfo, unauthorizedForWriteRequestInfo) = existingAndAuthorizedForDescribeTopics.partition { - case (topicPartition, _) => authorize(request.session, Write, new Resource(auth.Topic, topicPartition.topic)) + case (tp, _) => authorize(request.session, Write, new Resource(auth.Topic, tp.topic)) } // the callback for sending a produce response def sendResponseCallback(responseStatus: Map[TopicPartition, PartitionResponse]) { val mergedResponseStatus = responseStatus ++ - unauthorizedForWriteRequestInfo.mapValues(_ => - new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Record.NO_TIMESTAMP)) ++ - nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => - new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Record.NO_TIMESTAMP)) + unauthorizedForWriteRequestInfo.mapValues(_ => new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED)) ++ + nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ => new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION)) var errorInResponse = false mergedResponseStatus.foreach { case (topicPartition, status) => - if (status.errorCode != Errors.NONE.code) { + if (status.error != Errors.NONE) { errorInResponse = true debug("Produce request with correlation id %d from client %s on partition %s failed due to %s".format( request.header.correlationId, request.header.clientId, topicPartition, - Errors.forCode(status.errorCode).exceptionName)) + status.error.exceptionName)) } } @@ -383,7 +385,7 @@ class KafkaApis(val requestChannel: RequestChannel, // the producer client will know that some error has happened and will refresh its metadata if (errorInResponse) { val exceptionsSummary = mergedResponseStatus.map { case (topicPartition, status) => - topicPartition -> Errors.forCode(status.errorCode).exceptionName + topicPartition -> status.error.exceptionName }.mkString(", ") info( s"Closing connection due to error during produce request with correlation id ${request.header.correlationId} " + @@ -430,9 +432,8 @@ class KafkaApis(val requestChannel: RequestChannel, authorizedRequestInfo, sendResponseCallback) - // if the request is put into the purgatory, it will have a held reference - // and hence cannot be garbage collected; hence we clear its data here in - // order to let GC re-claim its memory since it is already appended to log + // if the request is put into the purgatory, it will have a held reference and hence cannot be garbage collected; + // hence we clear its data here inorder to let GC re-claim its memory since it is already appended to log produceRequest.clearPartitionRecords() } } @@ -474,13 +475,13 @@ class KafkaApis(val requestChannel: RequestChannel, // Please note that if the message format is changed from a higher version back to lower version this // test might break because some messages in new message format can be delivered to consumers before 0.10.0.0 // without format down conversion. - val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) && + val convertedData = if (versionId <= 1 && replicaManager.getMagic(tp).exists(_ > Record.MAGIC_VALUE_V0) && !data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) { trace(s"Down converting message to V0 for fetch request from $clientId") FetchPartitionData(data.error, data.hw, data.records.toMessageFormat(Record.MAGIC_VALUE_V0)) } else data - tp -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records) + tp -> new FetchResponse.PartitionData(convertedData.error.code, convertedData.hw, convertedData.records) } } @@ -886,38 +887,31 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, responseBody)) } - /* + /** * Handle an offset fetch request */ def handleOffsetFetchRequest(request: RequestChannel.Request) { val header = request.header val offsetFetchRequest = request.body.asInstanceOf[OffsetFetchRequest] + def authorizeTopicDescribe(partition: TopicPartition) = + authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) + val offsetFetchResponse = // reject the request if not authorized to the group if (!authorize(request.session, Read, new Resource(Group, offsetFetchRequest.groupId))) - new OffsetFetchResponse(Errors.GROUP_AUTHORIZATION_FAILED, offsetFetchRequest.partitions, header.apiVersion) + offsetFetchRequest.getErrorResponse(Errors.GROUP_AUTHORIZATION_FAILED) else { - val partitions = - if (offsetFetchRequest.isAllPartitions) - List[TopicPartition]() - else - offsetFetchRequest.partitions.asScala.toList - - val (authorizedPartitions, unauthorizedPartitions) = - partitions.partition { partition => authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) } - - val unknownTopicPartitionResponse = new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, Errors.UNKNOWN_TOPIC_OR_PARTITION) - val unauthorizedStatus = unauthorizedPartitions.map(topicPartition => (topicPartition, unknownTopicPartitionResponse)).toMap - if (header.apiVersion == 0) { + val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala + .partition(authorizeTopicDescribe) + // version 0 reads offsets from ZK - val responseInfo = authorizedPartitions.map { topicPartition => + val authorizedPartitionData = authorizedPartitions.map { topicPartition => val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, topicPartition.topic) try { if (!metadataCache.contains(topicPartition.topic)) - (topicPartition, unknownTopicPartitionResponse) + (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION) else { val payloadOpt = zkUtils.readDataMaybeNull(s"${topicDirs.consumerOffsetDir}/${topicPartition.partition}")._1 payloadOpt match { @@ -925,7 +919,7 @@ class KafkaApis(val requestChannel: RequestChannel, (topicPartition, new OffsetFetchResponse.PartitionData( payload.toLong, OffsetFetchResponse.NO_METADATA, Errors.NONE)) case None => - (topicPartition, unknownTopicPartitionResponse) + (topicPartition, OffsetFetchResponse.UNKNOWN_PARTITION) } } } catch { @@ -934,43 +928,32 @@ class KafkaApis(val requestChannel: RequestChannel, OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, Errors.forException(e))) } }.toMap - new OffsetFetchResponse(Errors.NONE, (responseInfo ++ unauthorizedStatus).asJava, header.apiVersion) - } - else { + + val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) + } else { // versions 1 and above read offsets from Kafka - val offsets = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, - if (offsetFetchRequest.isAllPartitions) - None - else + if (offsetFetchRequest.isAllPartitions) { + val (error, allPartitionData) = coordinator.handleFetchOffsets(offsetFetchRequest.groupId) + if (error != Errors.NONE) + offsetFetchRequest.getErrorResponse(error) + else { + // clients are not allowed to see offsets for topics that are not authorized for Describe + val authorizedPartitionData = allPartitionData.filter { case (topicPartition, _) => authorizeTopicDescribe(topicPartition) } + new OffsetFetchResponse(Errors.NONE, authorizedPartitionData.asJava, header.apiVersion) + } + } else { + val (authorizedPartitions, unauthorizedPartitions) = offsetFetchRequest.partitions.asScala + .partition(authorizeTopicDescribe) + val (error, authorizedPartitionData) = coordinator.handleFetchOffsets(offsetFetchRequest.groupId, Some(authorizedPartitions)) - - // Note that we do not need to filter the partitions in the - // metadata cache as the topic partitions will be filtered - // in coordinator's offset manager through the offset cache - if (header.apiVersion == 1) { - val authorizedStatus = - if (offsets._1 != Errors.NONE) { - authorizedPartitions.map { partition => - (partition, new OffsetFetchResponse.PartitionData( - OffsetFetchResponse.INVALID_OFFSET, OffsetFetchResponse.NO_METADATA, offsets._1))}.toMap - } - else - offsets._2.toMap - new OffsetFetchResponse(Errors.NONE, (authorizedStatus ++ unauthorizedStatus).asJava, header.apiVersion) - } - else if (offsets._1 == Errors.NONE) { - if (offsetFetchRequest.isAllPartitions) { - // filter out unauthorized topics in case all group offsets are requested - val authorizedStatus = offsets._2.filter { - case (partition, _) => authorize(request.session, Describe, new Resource(auth.Topic, partition.topic)) - } - new OffsetFetchResponse((authorizedStatus).asJava) + if (error != Errors.NONE) + offsetFetchRequest.getErrorResponse(error) + else { + val unauthorizedPartitionData = unauthorizedPartitions.map(_ -> OffsetFetchResponse.UNKNOWN_PARTITION).toMap + new OffsetFetchResponse(Errors.NONE, (authorizedPartitionData ++ unauthorizedPartitionData).asJava, header.apiVersion) } - else - new OffsetFetchResponse((offsets._2.toMap ++ unauthorizedStatus).asJava) } - else - new OffsetFetchResponse(offsets._1) } } diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 891327f605d21..257754308020f 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -281,7 +281,7 @@ object KafkaConfig { val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" - val CreateTopicsPolicyClassNameProp = "create.topics.policy.class.name" + val CreateTopicPolicyClassNameProp = "create.topic.policy.class.name" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -491,7 +491,7 @@ object KafkaConfig { "produce with acks of \"all\". This will ensure that the producer raises an exception " + "if a majority of replicas do not receive a write." - val CreateTopicsPolicyClassNameDoc = "The create topics policy class that should be used for validation. The class should " + + val CreateTopicPolicyClassNameDoc = "The create topic policy class that should be used for validation. The class should " + "implement the org.apache.kafka.server.policy.CreateTopicPolicy interface." /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" @@ -693,7 +693,7 @@ object KafkaConfig { .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, LogMessageFormatVersionDoc) .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) - .define(CreateTopicsPolicyClassNameProp, CLASS, null, LOW, CreateTopicsPolicyClassNameDoc) + .define(CreateTopicPolicyClassNameProp, CLASS, null, LOW, CreateTopicPolicyClassNameDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) @@ -824,6 +824,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val brokerIdGenerationEnable: Boolean = getBoolean(KafkaConfig.BrokerIdGenerationEnableProp) val maxReservedBrokerId: Int = getInt(KafkaConfig.MaxReservedBrokerIdProp) var brokerId: Int = getInt(KafkaConfig.BrokerIdProp) + val numNetworkThreads = getInt(KafkaConfig.NumNetworkThreadsProp) val backgroundThreads = getInt(KafkaConfig.BackgroundThreadsProp) val queuedMaxRequests = getInt(KafkaConfig.QueuedMaxRequestsProp) @@ -940,7 +941,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra /** ********* Metric Configuration **************/ val metricNumSamples = getInt(KafkaConfig.MetricNumSamplesProp) val metricSampleWindowMs = getLong(KafkaConfig.MetricSampleWindowMsProp) - val metricReporterClasses: java.util.List[MetricsReporter] = getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter]) + val metricRecordingLevel = getString(KafkaConfig.MetricRecordingLevelProp) /** ********* SSL Configuration **************/ val principalBuilderClass = getClass(KafkaConfig.PrincipalBuilderClassProp) diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 79548e86c5969..c773382683a83 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -79,9 +79,15 @@ object KafkaServer { logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs) logProps } -} + private[server] def metricConfig(kafkaConfig: KafkaConfig): MetricConfig = { + new MetricConfig() + .samples(kafkaConfig.metricNumSamples) + .recordLevel(Sensor.RecordingLevel.forName(kafkaConfig.metricRecordingLevel)) + .timeWindow(kafkaConfig.metricSampleWindowMs, TimeUnit.MILLISECONDS) + } +} /** * Represents the lifecycle of a single Kafka broker. Handles all functionality required @@ -95,15 +101,9 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP private var shutdownLatch = new CountDownLatch(1) private val jmxPrefix: String = "kafka.server" - private val reporters: java.util.List[MetricsReporter] = config.metricReporterClasses - reporters.add(new JmxReporter(jmxPrefix)) var metrics: Metrics = null - private val metricConfig: MetricConfig = new MetricConfig() - .samples(config.metricNumSamples) - .timeWindow(config.metricSampleWindowMs, TimeUnit.MILLISECONDS) - val brokerState: BrokerState = new BrokerState var apis: KafkaApis = null @@ -178,9 +178,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP val canStartup = isStartingUp.compareAndSet(false, true) if (canStartup) { - metrics = new Metrics(metricConfig, reporters, time, true) - quotaManagers = QuotaFactory.instantiate(config, metrics, time) - brokerState.newState(Starting) /* start scheduler */ @@ -193,16 +190,24 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP _clusterId = getOrGenerateClusterId(zkUtils) info(s"Cluster ID = $clusterId") + /* generate brokerId */ + config.brokerId = getBrokerId + this.logIdent = "[Kafka Server " + config.brokerId + "], " + + /* create and configure metrics */ + val reporters = config.getConfiguredInstances(KafkaConfig.MetricReporterClassesProp, classOf[MetricsReporter], + Map[String, AnyRef](KafkaConfig.BrokerIdProp -> (config.brokerId.toString)).asJava) + reporters.add(new JmxReporter(jmxPrefix)) + val metricConfig = KafkaServer.metricConfig(config) + metrics = new Metrics(metricConfig, reporters, time, true) + + quotaManagers = QuotaFactory.instantiate(config, metrics, time) notifyClusterListeners(kafkaMetricsReporters ++ reporters.asScala) /* start log manager */ logManager = createLogManager(zkUtils.zkClient, brokerState) logManager.startup() - /* generate brokerId */ - config.brokerId = getBrokerId - this.logIdent = "[Kafka Server " + config.brokerId + "], " - metadataCache = new MetadataCache(config.brokerId) credentialProvider = new CredentialProvider(config.saslEnabledMechanisms) @@ -569,8 +574,10 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP if(isStartingUp.get) throw new IllegalStateException("Kafka server is still starting up, cannot shut down!") - val canShutdown = isShuttingDown.compareAndSet(false, true) - if (canShutdown && shutdownLatch.getCount > 0) { + // To ensure correct behavior under concurrent calls, we need to check `shutdownLatch` first since it gets updated + // last in the `if` block. If the order is reversed, we could shutdown twice or leave `isShuttingDown` set to + // `true` at the end of this method. + if (shutdownLatch.getCount > 0 && isShuttingDown.compareAndSet(false, true)) { CoreUtils.swallow(controlledShutdown()) brokerState.newState(BrokerShuttingDown) if(socketServer != null) @@ -600,7 +607,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = Time.SYSTEM, threadNameP startupComplete.set(false) isShuttingDown.set(false) - AppInfoParser.unregisterAppInfo(jmxPrefix, config.brokerId.toString) + CoreUtils.swallow(AppInfoParser.unregisterAppInfo(jmxPrefix, config.brokerId.toString)) shutdownLatch.countDown() info("shut down completed") } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 5c28e140808e5..9a6090d3647d6 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -27,7 +27,7 @@ import kafka.common.{BrokerEndPointNotAvailableException, Topic, TopicAndPartiti import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch} import kafka.utils.CoreUtils._ import kafka.utils.Logging -import org.apache.kafka.common.Node +import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.requests.{MetadataResponse, PartitionState, UpdateMetadataRequest} @@ -155,7 +155,8 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { def getControllerId: Option[Int] = controllerId - def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest) { + // This method returns the deleted TopicPartitions received from UpdateMetadataRequest + def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest): Seq[TopicPartition] = { inWriteLock(partitionMetadataLock) { controllerId = updateMetadataRequest.controllerId match { case id if id < 0 => None @@ -177,6 +178,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { aliveNodes(broker.id) = nodes.asScala } + val deletedPartitions = new mutable.ArrayBuffer[TopicPartition] updateMetadataRequest.partitionStates.asScala.foreach { case (tp, info) => val controllerId = updateMetadataRequest.controllerId val controllerEpoch = updateMetadataRequest.controllerEpoch @@ -184,6 +186,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { removePartitionInfo(tp.topic, tp.partition) stateChangeLogger.trace(s"Broker $brokerId deleted partition $tp from metadata cache in response to UpdateMetadata " + s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") + deletedPartitions += tp } else { val partitionInfo = partitionStateToPartitionStateInfo(info) addOrUpdatePartitionInfo(tp.topic, tp.partition, partitionInfo) @@ -191,6 +194,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { s"UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") } } + deletedPartitions } } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 199bb67482ba8..040cab6eaa023 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -25,7 +25,7 @@ import kafka.api._ import kafka.cluster.{Partition, Replica} import kafka.common._ import kafka.controller.KafkaController -import kafka.log.{LogAppendInfo, LogManager} +import kafka.log.{Log, LogAppendInfo, LogManager} import kafka.metrics.KafkaMetricsGroup import kafka.server.QuotaFactory.UnboundedQuota import kafka.utils._ @@ -45,10 +45,10 @@ import scala.collection.JavaConverters._ /* * Result metadata of a log append operation on the log */ -case class LogAppendResult(info: LogAppendInfo, error: Option[Throwable] = None) { - def errorCode = error match { - case None => Errors.NONE.code - case Some(e) => Errors.forException(e).code +case class LogAppendResult(info: LogAppendInfo, exception: Option[Throwable] = None) { + def error: Errors = exception match { + case None => Errors.NONE + case Some(e) => Errors.forException(e) } } @@ -66,20 +66,19 @@ case class LogReadResult(info: FetchDataInfo, leaderLogEndOffset: Long, fetchTimeMs: Long, readSize: Int, - error: Option[Throwable] = None) { + exception: Option[Throwable] = None) { - def errorCode = error match { - case None => Errors.NONE.code - case Some(e) => Errors.forException(e).code + def error: Errors = exception match { + case None => Errors.NONE + case Some(e) => Errors.forException(e) } - override def toString = { - "Fetch Data: [%s], HW: [%d], leaderLogEndOffset: [%d], readSize: [%d], error: [%s]" - .format(info, hw, leaderLogEndOffset, readSize, error) - } + override def toString = + s"Fetch Data: [$info], HW: [$hw], leaderLogEndOffset: [$leaderLogEndOffset], readSize: [$readSize], error: [$error]" + } -case class FetchPartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, records: Records) +case class FetchPartitionData(error: Errors = Errors.NONE, hw: Long = -1L, records: Records) object LogReadResult { val UnknownLogReadResult = LogReadResult(info = FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), @@ -190,6 +189,8 @@ class ReplicaManager(val config: KafkaConfig, } } + def getLog(topicPartition: TopicPartition): Option[Log] = logManager.getLog(topicPartition) + /** * Try to complete some delayed produce requests with the request key; * this can be triggered when: @@ -315,7 +316,7 @@ class ReplicaManager(val config: KafkaConfig, topicPartition -> ProducePartitionStatus( result.info.lastOffset + 1, // required offset - new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status + new PartitionResponse(result.error, result.info.firstOffset, result.info.logAppendTime)) // response status } if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) { @@ -340,7 +341,7 @@ class ReplicaManager(val config: KafkaConfig, // If required.acks is outside accepted range, something is wrong with the client // Just return an error and don't handle the request at all val responseStatus = entriesPerPartition.map { case (topicPartition, _) => - topicPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code, + topicPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS, LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP) } responseCallback(responseStatus) @@ -357,7 +358,7 @@ class ReplicaManager(val config: KafkaConfig, localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = { requiredAcks == -1 && entriesPerPartition.nonEmpty && - localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size + localProduceResults.values.count(_.exception.isDefined) < entriesPerPartition.size } private def isValidRequiredAcks(requiredAcks: Short): Boolean = { @@ -464,7 +465,7 @@ class ReplicaManager(val config: KafkaConfig, val logReadResultValues = logReadResults.map { case (_, v) => v } val bytesReadable = logReadResultValues.map(_.info.records.sizeInBytes).sum val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) => - errorIncurred || (readResult.errorCode != Errors.NONE.code)) + errorIncurred || (readResult.error != Errors.NONE)) // respond immediately if 1) fetch request does not want to wait // 2) fetch request does not require any data @@ -472,7 +473,7 @@ class ReplicaManager(val config: KafkaConfig, // 4) some error happens while reading data if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) { val fetchPartitionData = logReadResults.map { case (tp, result) => - tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records) + tp -> FetchPartitionData(result.error, result.hw, result.info.records) } responseCallback(fetchPartitionData) } else { @@ -567,7 +568,7 @@ class ReplicaManager(val config: KafkaConfig, leaderLogEndOffset = initialLogEndOffset, fetchTimeMs = fetchTimeMs, readSize = partitionFetchSize, - error = None) + exception = None) } catch { // NOTE: Failed fetch requests metric is not incremented for known exceptions since it // is supposed to indicate un-expected failure of a broker in handling a fetch request @@ -580,7 +581,7 @@ class ReplicaManager(val config: KafkaConfig, leaderLogEndOffset = -1L, fetchTimeMs = -1L, readSize = partitionFetchSize, - error = Some(e)) + exception = Some(e)) case e: Throwable => BrokerTopicStats.getBrokerTopicStats(tp.topic).failedFetchRequestRate.mark() BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark() @@ -590,7 +591,7 @@ class ReplicaManager(val config: KafkaConfig, leaderLogEndOffset = -1L, fetchTimeMs = -1L, readSize = partitionFetchSize, - error = Some(e)) + exception = Some(e)) } } @@ -620,12 +621,10 @@ class ReplicaManager(val config: KafkaConfig, quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync } - def getMagicAndTimestampType(topicPartition: TopicPartition): Option[(Byte, TimestampType)] = - getReplica(topicPartition).flatMap { replica => - replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType)) - } + def getMagic(topicPartition: TopicPartition): Option[Byte] = + getReplica(topicPartition).flatMap(_.log.map(_.config.messageFormatVersion.messageFormatVersion)) - def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) { + def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) : Seq[TopicPartition] = { replicaStateChangeLock synchronized { if(updateMetadataRequest.controllerEpoch < controllerEpoch) { val stateControllerEpochErrorMessage = ("Broker %d received update metadata request with correlation id %d from an " + @@ -635,8 +634,9 @@ class ReplicaManager(val config: KafkaConfig, stateChangeLogger.warn(stateControllerEpochErrorMessage) throw new ControllerMovedException(stateControllerEpochErrorMessage) } else { - metadataCache.updateCache(correlationId, updateMetadataRequest) + val deletedPartitions = metadataCache.updateCache(correlationId, updateMetadataRequest) controllerEpoch = updateMetadataRequest.controllerEpoch + deletedPartitions } } } @@ -926,10 +926,16 @@ class ReplicaManager(val config: KafkaConfig, } } - private def getLeaderPartitions() : List[Partition] = { + private def getLeaderPartitions(): List[Partition] = { allPartitions.values.filter(_.leaderReplicaIfLocal.isDefined).toList } + def getHighWatermark(topicPartition: TopicPartition): Option[Long] = { + getPartition(topicPartition).flatMap { partition => + partition.leaderReplicaIfLocal.map(_.highWatermark.messageOffset) + } + } + // Flushes the highwatermark value for all partitions to the highwatermark file def checkpointHighWatermarks() { val replicas = allPartitions.values.flatMap(_.getReplica(localBrokerId)) diff --git a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala old mode 100755 new mode 100644 index ca0f6a03d8eb5..f41782e5a460b --- a/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala +++ b/core/src/main/scala/kafka/server/ZookeeperLeaderElector.scala @@ -52,7 +52,7 @@ class ZookeeperLeaderElector(controllerContext: ControllerContext, } } - private def getControllerID(): Int = { + def getControllerID(): Int = { controllerContext.zkUtils.readDataMaybeNull(electionPath)._1 match { case Some(controller) => KafkaController.parseControllerId(controller) case None => -1 diff --git a/core/src/main/scala/kafka/tools/JmxTool.scala b/core/src/main/scala/kafka/tools/JmxTool.scala index 3538874d5f2d9..5a672aaebfcce 100644 --- a/core/src/main/scala/kafka/tools/JmxTool.scala +++ b/core/src/main/scala/kafka/tools/JmxTool.scala @@ -28,6 +28,13 @@ import scala.collection.mutable import scala.math._ import kafka.utils.{CommandLineUtils, Logging} + +/** + * A program for reading JMX metrics from a given endpoint. + * + * This tool only works reliably if the JmxServer is fully initialized prior to invoking the tool. See KAFKA-4620 for + * details. + */ object JmxTool extends Logging { def main(args: Array[String]) { @@ -80,8 +87,32 @@ object JmxTool extends Logging { val attributesWhitelist = if(attributesWhitelistExists) Some(options.valueOf(attributesOpt).split(",")) else None val dateFormatExists = options.has(dateFormatOpt) val dateFormat = if(dateFormatExists) Some(new SimpleDateFormat(options.valueOf(dateFormatOpt))) else None - val jmxc = JMXConnectorFactory.connect(url, null) - val mbsc = jmxc.getMBeanServerConnection() + + var jmxc: JMXConnector = null + var mbsc: MBeanServerConnection = null + var retries = 0 + val maxNumRetries = 10 + var connected = false + while (retries < maxNumRetries && !connected) { + try { + System.err.println(s"Trying to connect to JMX url: $url.") + jmxc = JMXConnectorFactory.connect(url, null) + mbsc = jmxc.getMBeanServerConnection + connected = true + } catch { + case e : Exception => + System.err.println(s"Could not connect to JMX url: $url. Exception ${e.getMessage}.") + e.printStackTrace() + retries += 1 + Thread.sleep(500) + } + } + + if (!connected) { + System.err.println(s"Could not connect to JMX url $url after $maxNumRetries retries.") + System.err.println("Exiting.") + sys.exit(1) + } val queries: Iterable[ObjectName] = if(options.has(objectNameOpt)) diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 7a6bd6336b886..648ac60b8e8b7 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -43,19 +43,46 @@ import scala.collection._ import scala.collection.JavaConverters._ object ZkUtils { - val ConsumersPath = "/consumers" - val ClusterIdPath = "/cluster/id" - val BrokerIdsPath = "/brokers/ids" - val BrokerTopicsPath = "/brokers/topics" + + private val UseDefaultAcls = new java.util.ArrayList[ACL] + + // Important: it is necessary to add any new top level Zookeeper path here + val AdminPath = "/admin" + val BrokersPath = "/brokers" + val ClusterPath = "/cluster" + val ConfigPath = "/config" val ControllerPath = "/controller" val ControllerEpochPath = "/controller_epoch" - val ReassignPartitionsPath = "/admin/reassign_partitions" - val DeleteTopicsPath = "/admin/delete_topics" - val PreferredReplicaLeaderElectionPath = "/admin/preferred_replica_election" - val BrokerSequenceIdPath = "/brokers/seqid" val IsrChangeNotificationPath = "/isr_change_notification" - val EntityConfigPath = "/config" - val EntityConfigChangesPath = "/config/changes" + val KafkaAclPath = "/kafka-acl" + val KafkaAclChangesPath = "/kafka-acl-changes" + + val ConsumersPath = "/consumers" + val ClusterIdPath = s"$ClusterPath/id" + val BrokerIdsPath = s"$BrokersPath/ids" + val BrokerTopicsPath = s"$BrokersPath/topics" + val ReassignPartitionsPath = s"$AdminPath/reassign_partitions" + val DeleteTopicsPath = s"$AdminPath/delete_topics" + val PreferredReplicaLeaderElectionPath = s"$AdminPath/preferred_replica_election" + val BrokerSequenceIdPath = s"$BrokersPath/seqid" + val ConfigChangesPath = s"$ConfigPath/changes" + val ConfigUsersPath = s"$ConfigPath/users" + + + // Important: it is necessary to add any new top level Zookeeper path to the Seq + val SecureZkRootPaths = Seq(AdminPath, + BrokersPath, + ClusterPath, + ConfigPath, + ControllerPath, + ControllerEpochPath, + IsrChangeNotificationPath, + KafkaAclPath, + KafkaAclChangesPath) + + // Important: it is necessary to add any new top level Zookeeper path that contains + // sensitive information that should not be world readable to the Seq + val SensitiveZkRootPaths = Seq(ConfigUsersPath) def apply(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int, isZkSecurityEnabled: Boolean): ZkUtils = { val (zkClient, zkConnection) = createZkClientAndConnection(zkUrl, sessionTimeout, connectionTimeout) @@ -80,13 +107,23 @@ object ZkUtils { (zkClient, zkConnection) } - def DefaultAcls(isSecure: Boolean): java.util.List[ACL] = if (isSecure) { - val list = new java.util.ArrayList[ACL] - list.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) - list.addAll(ZooDefs.Ids.READ_ACL_UNSAFE) - list - } else { - ZooDefs.Ids.OPEN_ACL_UNSAFE + def sensitivePath(path: String): Boolean = { + path != null && SensitiveZkRootPaths.exists(path.startsWith(_)) + } + + @deprecated("This is deprecated, use defaultAcls(isSecure, path) which doesn't make sensitive data world readable", since = "0.10.2.1") + def DefaultAcls(isSecure: Boolean): java.util.List[ACL] = defaultAcls(isSecure, "") + + def defaultAcls(isSecure: Boolean, path: String): java.util.List[ACL] = { + if (isSecure) { + val list = new java.util.ArrayList[ACL] + list.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) + if (!sensitivePath(path)) { + list.addAll(ZooDefs.Ids.READ_ACL_UNSAFE) + } + list + } else + ZooDefs.Ids.OPEN_ACL_UNSAFE } def maybeDeletePath(zkUrl: String, dir: String) { @@ -117,13 +154,13 @@ object ZkUtils { getTopicPartitionPath(topic, partitionId) + "/" + "state" def getEntityConfigRootPath(entityType: String): String = - ZkUtils.EntityConfigPath + "/" + entityType + ZkUtils.ConfigPath + "/" + entityType def getEntityConfigPath(entityType: String, entity: String): String = getEntityConfigRootPath(entityType) + "/" + entity def getEntityConfigPath(entityPath: String): String = - ZkUtils.EntityConfigPath + "/" + entityPath + ZkUtils.ConfigPath + "/" + entityPath def getDeleteTopicPath(topic: String): String = DeleteTopicsPath + "/" + topic @@ -191,23 +228,19 @@ class ZkUtils(val zkClient: ZkClient, val persistentZkPaths = Seq(ConsumersPath, BrokerIdsPath, BrokerTopicsPath, - EntityConfigChangesPath, + ConfigChangesPath, getEntityConfigRootPath(ConfigType.Topic), getEntityConfigRootPath(ConfigType.Client), DeleteTopicsPath, BrokerSequenceIdPath, IsrChangeNotificationPath) - val securePersistentZkPaths = Seq(BrokerIdsPath, - BrokerTopicsPath, - EntityConfigChangesPath, - getEntityConfigRootPath(ConfigType.Topic), - getEntityConfigRootPath(ConfigType.Client), - DeleteTopicsPath, - BrokerSequenceIdPath, - IsrChangeNotificationPath) + import ZkUtils._ - val DefaultAcls: java.util.List[ACL] = ZkUtils.DefaultAcls(isSecure) + @deprecated("This is deprecated, use defaultAcls(path) which doesn't make sensitive data world readable", since = "0.10.2.1") + val DefaultAcls: java.util.List[ACL] = ZkUtils.defaultAcls(isSecure, "") + + def defaultAcls(path: String): java.util.List[ACL] = ZkUtils.defaultAcls(isSecure, path) def getController(): Int = { readDataMaybeNull(ControllerPath)._1 match { @@ -400,11 +433,15 @@ class ZkUtils(val zkClient: ZkClient, /** * make sure a persistent path exists in ZK. Create the path if not exist. */ - def makeSurePersistentPathExists(path: String, acls: java.util.List[ACL] = DefaultAcls) { + def makeSurePersistentPathExists(path: String, acls: java.util.List[ACL] = UseDefaultAcls) { //Consumer path is kept open as different consumers will write under this node. val acl = if (path == null || path.isEmpty || path.equals(ConsumersPath)) { ZooDefs.Ids.OPEN_ACL_UNSAFE - } else acls + } else if (acls eq UseDefaultAcls) { + ZkUtils.defaultAcls(isSecure, path) + } else { + acls + } if (!zkClient.exists(path)) ZkPath.createPersistent(zkClient, path, createParents = true, acl) //won't throw NoNodeException or NodeExistsException @@ -413,23 +450,25 @@ class ZkUtils(val zkClient: ZkClient, /** * create the parent path */ - private def createParentPath(path: String, acls: java.util.List[ACL] = DefaultAcls): Unit = { + private def createParentPath(path: String, acls: java.util.List[ACL] = UseDefaultAcls): Unit = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls val parentDir = path.substring(0, path.lastIndexOf('/')) if (parentDir.length != 0) { - ZkPath.createPersistent(zkClient, parentDir, createParents = true, acls) + ZkPath.createPersistent(zkClient, parentDir, createParents = true, acl) } } /** * Create an ephemeral node with the given path and data. Create parents if necessary. */ - private def createEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = { + private def createEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = UseDefaultAcls): Unit = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls try { - ZkPath.createEphemeral(zkClient, path, data, acls) + ZkPath.createEphemeral(zkClient, path, data, acl) } catch { case _: ZkNoNodeException => createParentPath(path) - ZkPath.createEphemeral(zkClient, path, data, acls) + ZkPath.createEphemeral(zkClient, path, data, acl) } } @@ -437,9 +476,10 @@ class ZkUtils(val zkClient: ZkClient, * Create an ephemeral node with the given path and data. * Throw NodeExistException if node already exists. */ - def createEphemeralPathExpectConflict(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = { + def createEphemeralPathExpectConflict(path: String, data: String, acls: java.util.List[ACL] = UseDefaultAcls): Unit = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls try { - createEphemeralPath(path, data, acls) + createEphemeralPath(path, data, acl) } catch { case e: ZkNodeExistsException => // this can happen when there is connection loss; make sure the data is what we intend to write @@ -462,18 +502,20 @@ class ZkUtils(val zkClient: ZkClient, /** * Create an persistent node with the given path and data. Create parents if necessary. */ - def createPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = DefaultAcls): Unit = { + def createPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = UseDefaultAcls): Unit = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls try { - ZkPath.createPersistent(zkClient, path, data, acls) + ZkPath.createPersistent(zkClient, path, data, acl) } catch { case _: ZkNoNodeException => createParentPath(path) - ZkPath.createPersistent(zkClient, path, data, acls) + ZkPath.createPersistent(zkClient, path, data, acl) } } - def createSequentialPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = DefaultAcls): String = { - ZkPath.createPersistentSequential(zkClient, path, data, acls) + def createSequentialPersistentPath(path: String, data: String = "", acls: java.util.List[ACL] = UseDefaultAcls): String = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls + ZkPath.createPersistentSequential(zkClient, path, data, acl) } /** @@ -481,14 +523,15 @@ class ZkUtils(val zkClient: ZkClient, * create parent directory if necessary. Never throw NodeExistException. * Return the updated path zkVersion */ - def updatePersistentPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls) = { + def updatePersistentPath(path: String, data: String, acls: java.util.List[ACL] = UseDefaultAcls) = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls try { zkClient.writeData(path, data) } catch { case _: ZkNoNodeException => createParentPath(path) try { - ZkPath.createPersistent(zkClient, path, data, acls) + ZkPath.createPersistent(zkClient, path, data, acl) } catch { case _: ZkNodeExistsException => zkClient.writeData(path, data) @@ -551,13 +594,14 @@ class ZkUtils(val zkClient: ZkClient, * Update the value of a persistent node with the given path and data. * create parent directory if necessary. Never throw NodeExistException. */ - def updateEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = DefaultAcls): Unit = { + def updateEphemeralPath(path: String, data: String, acls: java.util.List[ACL] = UseDefaultAcls): Unit = { + val acl = if (acls eq UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls try { zkClient.writeData(path, data) } catch { case _: ZkNoNodeException => createParentPath(path) - ZkPath.createEphemeral(zkClient, path, data, acls) + ZkPath.createEphemeral(zkClient, path, data, acl) } } @@ -806,13 +850,14 @@ class ZkUtils(val zkClient: ZkClient, * It uses the stat returned by the zookeeper and return the version. Every time * client updates the path stat.version gets incremented. Starting value of sequence number is 1. */ - def getSequenceId(path: String, acls: java.util.List[ACL] = DefaultAcls): Int = { + def getSequenceId(path: String, acls: java.util.List[ACL] = UseDefaultAcls): Int = { + val acl = if (acls == UseDefaultAcls) ZkUtils.defaultAcls(isSecure, path) else acls def writeToZk: Int = zkClient.writeDataReturnStat(path, "", -1).getVersion try { writeToZk } catch { case _: ZkNoNodeException => - makeSurePersistentPathExists(path) + makeSurePersistentPathExists(path, acl) writeToZk } } @@ -978,6 +1023,7 @@ class ZKCheckedEphemeral(path: String, private val getDataCallback = new GetDataCallback val latch: CountDownLatch = new CountDownLatch(1) var result: Code = Code.OK + val defaultAcls = ZkUtils.defaultAcls(isSecure, path) private class CreateCallback extends StringCallback { def processResult(rc: Int, @@ -1039,7 +1085,7 @@ class ZKCheckedEphemeral(path: String, private def createEphemeral() { zkHandle.create(path, ZKStringSerializer.serialize(data), - DefaultAcls(isSecure), + defaultAcls, CreateMode.EPHEMERAL, createCallback, null) @@ -1052,7 +1098,7 @@ class ZKCheckedEphemeral(path: String, } else { zkHandle.create(prefix, new Array[Byte](0), - DefaultAcls(isSecure), + defaultAcls, CreateMode.PERSISTENT, new StringCallback() { def processResult(rc : Int, diff --git a/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala new file mode 100644 index 0000000000000..8fa489d3fa331 --- /dev/null +++ b/core/src/test/scala/integration/kafka/admin/BrokerApiVersionsCommandTest.scala @@ -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 kafka.admin + +import java.io.{ByteArrayOutputStream, PrintStream} +import java.nio.charset.StandardCharsets + +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.NodeApiVersions +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.ApiVersionsResponse +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import org.junit.Test + +class BrokerApiVersionsCommandTest extends KafkaServerTestHarness { + + def generateConfigs(): Seq[KafkaConfig] = TestUtils.createBrokerConfigs(1, zkConnect).map(KafkaConfig.fromProps) + + @Test + def checkBrokerApiVersionCommandOutput() { + val byteArrayOutputStream = new ByteArrayOutputStream + val printStream = new PrintStream(byteArrayOutputStream) + BrokerApiVersionsCommand.execute(Array("--bootstrap-server", brokerList), printStream) + val content = new String(byteArrayOutputStream.toByteArray, StandardCharsets.UTF_8) + val lineIter = content.split("\n").iterator + assertTrue(lineIter.hasNext) + assertEquals(s"$brokerList (id: 0 rack: null) -> (", lineIter.next) + val nodeApiVersions = NodeApiVersions.create + for (apiKey <- ApiKeys.values) { + val apiVersion = nodeApiVersions.apiVersion(apiKey) + val versionRangeStr = + if (apiVersion.minVersion == apiVersion.maxVersion) apiVersion.minVersion.toString + else s"${apiVersion.minVersion} to ${apiVersion.maxVersion}" + val terminator = if (apiKey == ApiKeys.values.last) "" else "," + val usableVersion = nodeApiVersions.usableVersion(apiKey) + val line = s"\t${apiKey.name}(${apiKey.id}): $versionRangeStr [usable: $usableVersion]$terminator" + assertTrue(lineIter.hasNext) + assertEquals(line, lineIter.next) + } + assertTrue(lineIter.hasNext) + assertEquals(")", lineIter.next) + assertFalse(lineIter.hasNext) + } +} diff --git a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala index a62922caaf278..1e2749f6946ca 100644 --- a/core/src/test/scala/integration/kafka/api/AdminClientTest.scala +++ b/core/src/test/scala/integration/kafka/api/AdminClientTest.scala @@ -24,6 +24,7 @@ import kafka.utils.{Logging, TestUtils} 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.protocol.ApiKeys import org.junit.{Before, Test} import org.junit.Assert._ @@ -77,6 +78,24 @@ class AdminClientTest extends IntegrationTestHarness with Logging { assertEquals("consumer", group.protocolType) } + @Test + def testListAllBrokerVersionInfo() { + consumers.head.subscribe(Collections.singletonList(topic)) + TestUtils.waitUntilTrue(() => { + consumers.head.poll(0) + !consumers.head.assignment.isEmpty + }, "Expected non-empty assignment") + val brokerVersionInfos = client.listAllBrokerVersionInfo + val brokers = brokerList.split(",") + assertEquals(brokers.size, brokerVersionInfos.size) + for ((node, tryBrokerVersionInfo) <- brokerVersionInfos) { + val hostStr = s"${node.host}:${node.port}" + assertTrue(s"Unknown host:port pair $hostStr in brokerVersionInfos", brokers.contains(hostStr)) + val brokerVersionInfo = tryBrokerVersionInfo.get + assertEquals(0, brokerVersionInfo.usableVersion(ApiKeys.API_VERSIONS)) + } + } + @Test def testGetConsumerGroupSummary() { consumers.head.subscribe(Collections.singletonList(topic)) diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 42251fa8950cb..1ba3214de5af6 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -40,6 +40,7 @@ import scala.collection.mutable import scala.collection.mutable.Buffer import org.apache.kafka.common.KafkaException import kafka.admin.AdminUtils +import kafka.network.SocketServer import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.record.MemoryRecords @@ -106,7 +107,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { val RequestKeyToErrorCode = Map[ApiKeys, (Nothing) => Short]( ApiKeys.METADATA -> ((resp: requests.MetadataResponse) => resp.errors().asScala.find(_._1 == topic).getOrElse(("test", Errors.NONE))._2.code), - ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses().asScala.find(_._1 == tp).get._2.errorCode), + ApiKeys.PRODUCE -> ((resp: requests.ProduceResponse) => resp.responses().asScala.find(_._1 == tp).get._2.error.code), ApiKeys.FETCH -> ((resp: requests.FetchResponse) => resp.responseData().asScala.find(_._1 == tp).get._2.errorCode), ApiKeys.LIST_OFFSETS -> ((resp: requests.ListOffsetResponse) => resp.responseData().asScala.find(_._1 == tp).get._2.errorCode), ApiKeys.OFFSET_COMMIT -> ((resp: requests.OffsetCommitResponse) => resp.responseData().asScala.find(_._1 == tp).get._2), @@ -173,7 +174,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { producers.foreach(_.close()) consumers.foreach(_.wakeup()) consumers.foreach(_.close()) - removeAllAcls + removeAllAcls() super.tearDown() } @@ -604,6 +605,7 @@ class AuthorizerIntegrationTest extends BaseRequestTest { try { consumer.subscribe(Pattern.compile(".*"), new NoOpConsumerRebalanceListener) consumeRecords(consumer) + consumeRecords(consumer) Assert.fail("Expected TopicAuthorizationException") } catch { case _: TopicAuthorizationException => //expected @@ -711,6 +713,34 @@ class AuthorizerIntegrationTest extends BaseRequestTest { this.consumers.head.position(tp) } + @Test + def testFetchAllOffsetsTopicAuthorization() { + val offset = 15L + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), topicResource) + this.consumers.head.assign(List(tp).asJava) + this.consumers.head.commitSync(Map(tp -> new OffsetAndMetadata(offset)).asJava) + + removeAllAcls() + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) + + // send offset fetch requests directly since the consumer does not expose an API to do so + // note there's only one broker, so no need to lookup the group coordinator + + // without describe permission on the topic, we shouldn't be able to fetch offsets + val offsetFetchRequest = requests.OffsetFetchRequest.forAllPartitions(group) + var offsetFetchResponse = sendOffsetFetchRequest(offsetFetchRequest, anySocketServer) + assertEquals(Errors.NONE, offsetFetchResponse.error) + assertTrue(offsetFetchResponse.responseData.isEmpty) + + // now add describe permission on the topic and verify that the offset can be fetched + addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Describe)), topicResource) + offsetFetchResponse = sendOffsetFetchRequest(offsetFetchRequest, anySocketServer) + assertEquals(Errors.NONE, offsetFetchResponse.error) + assertTrue(offsetFetchResponse.responseData.containsKey(tp)) + assertEquals(offset, offsetFetchResponse.responseData.get(tp).offset) + } + @Test def testOffsetFetchTopicDescribe() { addAndVerifyAcls(Set(new Acl(KafkaPrincipal.ANONYMOUS, Allow, Acl.WildCardHost, Read)), groupResource) @@ -846,4 +876,10 @@ class AuthorizerIntegrationTest extends BaseRequestTest { } } + private def sendOffsetFetchRequest(request: requests.OffsetFetchRequest, + socketServer: SocketServer): requests.OffsetFetchResponse = { + val response = send(request, ApiKeys.OFFSET_FETCH, socketServer) + requests.OffsetFetchResponse.parse(response, request.version) + } + } diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 4bc1678d589d5..27b89d5d1faf0 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -13,13 +13,12 @@ package kafka.api import java.util -import java.util.Collections import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.serialization.ByteArrayDeserializer -import org.apache.kafka.common.{MetricName, PartitionInfo, TopicPartition} +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import kafka.utils.{Logging, ShutdownableThread, TestUtils} import kafka.common.Topic import kafka.server.KafkaConfig @@ -34,7 +33,7 @@ import org.apache.kafka.common.errors.WakeupException /** * Integration tests for the new consumer that cover basic usage as well as server failures */ -abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { +abstract class BaseConsumerTest extends IntegrationTestHarness { val epsilon = 0.1 val producerCount = 1 @@ -118,104 +117,6 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { assertEquals(1, listener.callsToRevoked) } - @Test - def testPerPartitionLagMetricsCleanUpWithSubscribe() { - val numMessages = 1000 - val topic2 = "topic2" - TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers) - // send some messages. - sendRecords(numMessages, tp) - // Test subscribe - // Create a consumer and consumer some messages. - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithSubscribe") - consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithSubscribe") - val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - try { - val listener0 = new TestConsumerReassignmentListener - consumer.subscribe(List(topic, topic2).asJava, listener0) - var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() - TestUtils.waitUntilTrue(() => { - records = consumer.poll(100) - !records.records(tp).isEmpty - }, "Consumer did not consume any message before timeout.") - assertEquals("should be assigned once", 1, listener0.callsToAssigned) - // Verify the metric exist. - val tags = Collections.singletonMap("client-id", "testPerPartitionLagMetricsCleanUpWithSubscribe") - val fetchLag0 = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) - assertNotNull(fetchLag0) - val expectedLag = numMessages - records.count - assertEquals(s"The lag should be $expectedLag", expectedLag, fetchLag0.value, epsilon) - - // Remove topic from subscription - consumer.subscribe(List(topic2).asJava, listener0) - TestUtils.waitUntilTrue(() => { - consumer.poll(100) - listener0.callsToAssigned >= 2 - }, "Expected rebalance did not occur.") - // Verify the metric has gone - assertNull(consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) - assertNull(consumer.metrics.get(new MetricName(tp2 + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) - } finally { - consumer.close() - } - } - - @Test - def testPerPartitionLagMetricsCleanUpWithAssign() { - val numMessages = 1000 - // Test assign - // send some messages. - sendRecords(numMessages, tp) - sendRecords(numMessages, tp2) - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithAssign") - consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithAssign") - val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - try { - consumer.assign(List(tp).asJava) - var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() - TestUtils.waitUntilTrue(() => { - records = consumer.poll(100) - !records.records(tp).isEmpty - }, "Consumer did not consume any message before timeout.") - // Verify the metric exist. - val tags = Collections.singletonMap("client-id", "testPerPartitionLagMetricsCleanUpWithAssign") - val fetchLag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) - assertNotNull(fetchLag) - val expectedLag = numMessages - records.count - assertEquals(s"The lag should be $expectedLag", expectedLag, fetchLag.value, epsilon) - - consumer.assign(List(tp2).asJava) - TestUtils.waitUntilTrue(() => !consumer.poll(100).isEmpty, "Consumer did not consume any message before timeout.") - assertNull(consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) - } finally { - consumer.close() - } - } - - @Test - def testPerPartitionLagWithMaxPollRecords() { - val numMessages = 1000 - val maxPollRecords = 10 - sendRecords(numMessages, tp) - consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") - consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) - val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) - consumer.assign(List(tp).asJava) - try { - var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() - TestUtils.waitUntilTrue(() => { - records = consumer.poll(100) - !records.isEmpty - }, "Consumer did not consume any message before timeout.") - val tags = Collections.singletonMap("client-id", "testPerPartitionLagWithMaxPollRecords") - val lag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) - assertEquals(s"The lag should be ${numMessages - records.count}", numMessages - records.count, lag.value, epsilon) - } finally { - consumer.close() - } - } - protected class TestConsumerReassignmentListener extends ConsumerRebalanceListener { var callsToAssigned = 0 var callsToRevoked = 0 diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 82409bbcd88fc..da3f6519abace 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -43,7 +43,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val numServers = 2 overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties).map(KafkaConfig.fromProps(_, overridingProps)) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties).map(KafkaConfig.fromProps(_, overridingProps)) } private var consumer: KafkaConsumer[Array[Byte], Array[Byte]] = _ @@ -69,7 +69,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { protected def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = { val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, retries = retries, lingerMs = lingerMs, props = props) + saslProperties = clientSaslProperties, retries = retries, lingerMs = lingerMs, props = props) registerProducer(producer) } @@ -96,7 +96,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { @Test def testSendOffset() { val producer = createProducer(brokerList) - val partition = new Integer(0) + val partition = 0 object callback extends Callback { var offset = 0L @@ -175,8 +175,33 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { sendAndVerifyTimestamp(producer, TimestampType.CREATE_TIME) } + protected def sendAndVerify(producer: KafkaProducer[Array[Byte], Array[Byte]], + numRecords: Int = numRecords, + timeoutMs: Long = 20000L) { + val partition = 0 + try { + TestUtils.createTopic(zkUtils, topic, 1, 2, servers) + + val recordAndFutures = for (i <- 1 to numRecords) yield { + val record = new ProducerRecord(topic, partition, s"key$i".getBytes, s"value$i".getBytes) + (record, producer.send(record)) + } + producer.close(timeoutMs, TimeUnit.MILLISECONDS) + val lastOffset = recordAndFutures.foldLeft(0) { case (offset, (record, future)) => + val recordMetadata = future.get + assertEquals(topic, recordMetadata.topic) + assertEquals(partition, recordMetadata.partition) + assertEquals(offset, recordMetadata.offset) + offset + 1 + } + assertEquals(numRecords, lastOffset) + } finally { + producer.close() + } + } + protected def sendAndVerifyTimestamp(producer: KafkaProducer[Array[Byte], Array[Byte]], timestampType: TimestampType) { - val partition = new Integer(0) + val partition = 0 val baseTimestamp = 123456L val startTime = System.currentTimeMillis() @@ -187,13 +212,13 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { def onCompletion(metadata: RecordMetadata, exception: Exception) { if (exception == null) { - assertEquals(offset, metadata.offset()) - assertEquals(topic, metadata.topic()) + assertEquals(offset, metadata.offset) + assertEquals(topic, metadata.topic) if (timestampType == TimestampType.CREATE_TIME) - assertEquals(baseTimestamp + timestampDiff, metadata.timestamp()) + assertEquals(baseTimestamp + timestampDiff, metadata.timestamp) else - assertTrue(metadata.timestamp() >= startTime && metadata.timestamp() <= System.currentTimeMillis()) - assertEquals(partition, metadata.partition()) + assertTrue(metadata.timestamp >= startTime && metadata.timestamp <= System.currentTimeMillis()) + assertEquals(partition, metadata.partition) offset += 1 timestampDiff += 1 } else { @@ -211,11 +236,18 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { topicProps.setProperty(LogConfig.MessageTimestampTypeProp, "CreateTime") TestUtils.createTopic(zkUtils, topic, 1, 2, servers, topicProps) - for (i <- 1 to numRecords) { - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, partition, baseTimestamp + i, "key".getBytes, "value".getBytes) - producer.send(record, callback) + val recordAndFutures = for (i <- 1 to numRecords) yield { + val record = new ProducerRecord(topic, partition, baseTimestamp + i, s"key$i".getBytes, s"value$i".getBytes) + (record, producer.send(record, callback)) } producer.close(20000L, TimeUnit.MILLISECONDS) + recordAndFutures.foreach { case (record, future) => + val recordMetadata = future.get + if (timestampType == TimestampType.LOG_APPEND_TIME) + assertTrue(recordMetadata.timestamp >= startTime && recordMetadata.timestamp <= System.currentTimeMillis()) + else + assertEquals(record.timestamp, recordMetadata.timestamp) + } assertEquals(s"Should have offset $numRecords but only successfully sent ${callback.offset}", numRecords, callback.offset) } finally { producer.close() diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index f98716a503774..4ec77a153c8ba 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -13,8 +13,8 @@ package kafka.api +import java.util.concurrent._ import java.util.{Collection, Collections} -import java.util.concurrent.{Callable, Executors, ExecutorService, Future, Semaphore, TimeUnit} import kafka.admin.AdminClient import kafka.server.KafkaConfig @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord} import org.apache.kafka.common.TopicPartition import org.junit.Assert._ -import org.junit.{Before, After, Test} +import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ @@ -43,13 +43,13 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { // Time to process commit and leave group requests in tests when brokers are available val gracefulCloseTimeMs = 1000 - val executor = Executors.newFixedThreadPool(2) + val executor = Executors.newScheduledThreadPool(2) // configure the servers and clients - this.serverConfig.setProperty(KafkaConfig.ControlledShutdownEnableProp, "false") // speed up shutdown this.serverConfig.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "3") // don't want to lose offset this.serverConfig.setProperty(KafkaConfig.OffsetsTopicPartitionsProp, "1") this.serverConfig.setProperty(KafkaConfig.GroupMinSessionTimeoutMsProp, "10") // set small enough session timeout + this.serverConfig.setProperty(KafkaConfig.AutoCreateTopicsEnableProp, "false") this.producerConfig.setProperty(ProducerConfig.ACKS_CONFIG, "all") this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "my-test") this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 4096.toString) @@ -58,7 +58,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest") override def generateConfigs() = { - FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect,enableControlledShutdown = false) + FixedPortTestUtils.createBrokerConfigs(serverCount, zkConnect, enableControlledShutdown = false) .map(KafkaConfig.fromProps(_, serverConfig)) } @@ -161,6 +161,51 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { } } + @Test + def testSubscribeWhenTopicUnavailable() { + val numRecords = 1000 + val newtopic = "newtopic" + + val consumer = this.consumers.head + consumer.subscribe(Collections.singleton(newtopic)) + executor.schedule(new Runnable { + def run() = TestUtils.createTopic(zkUtils, newtopic, serverCount, serverCount, servers) + }, 2, TimeUnit.SECONDS) + consumer.poll(0) + + def sendRecords(numRecords: Int, topic: String = this.topic) { + var remainingRecords = numRecords + val endTimeMs = System.currentTimeMillis + 20000 + while (remainingRecords > 0 && System.currentTimeMillis < endTimeMs) { + val futures = (0 until remainingRecords).map { i => + this.producers.head.send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) + } + futures.map { future => + try { + future.get + remainingRecords -= 1 + } catch { + case _: Exception => + } + } + } + assertEquals(0, remainingRecords) + } + + sendRecords(numRecords, newtopic) + receiveRecords(consumer, numRecords, newtopic, 10000) + + servers.foreach(server => killBroker(server.config.brokerId)) + Thread.sleep(500) + restartDeadBrokers() + + val future = executor.submit(new Runnable { + def run() = receiveRecords(consumer, numRecords, newtopic, 10000) + }) + sendRecords(numRecords, newtopic) + future.get + } + @Test def testClose() { @@ -313,10 +358,12 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { consumer } - private def receiveRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int) { - var received = 0 - while (received < numRecords) + private def receiveRecords(consumer: KafkaConsumer[Array[Byte], Array[Byte]], numRecords: Int, topic: String = this.topic, timeoutMs: Long = 60000) { + var received = 0L + val endTimeMs = System.currentTimeMillis + timeoutMs + while (received < numRecords && System.currentTimeMillis < endTimeMs) received += consumer.poll(1000).count() + assertEquals(numRecords, received) } private def submitCloseAndValidate(consumer: KafkaConsumer[Array[Byte], Array[Byte]], @@ -374,7 +421,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { } } - private def sendRecords(numRecords: Int) { + private def sendRecords(numRecords: Int, topic: String = this.topic) { val futures = (0 until numRecords).map { i => this.producers.head.send(new ProducerRecord(topic, part, i.toString.getBytes, i.toString.getBytes)) } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 3e391d3911490..3866cc14e847a 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -169,7 +169,7 @@ abstract class EndToEndAuthorizationTest extends IntegrationTestHarness with Sas maxBlockMs = 3000L, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(producerConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala index 9e03e2781cf84..d885d9b2894f9 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndClusterIdTest.scala @@ -104,7 +104,7 @@ class EndToEndClusterIdTest extends KafkaServerTestHarness { override def generateConfigs() = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) cfgs.foreach(_.putAll(serverConfig)) cfgs.map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala index d15a01d055c21..bf5f8c1e6a3ec 100644 --- a/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala +++ b/core/src/test/scala/integration/kafka/api/FixedPortTestUtils.scala @@ -34,9 +34,7 @@ object FixedPortTestUtils { sockets.foreach(_.close()) ports } catch { - case e: IOException => { - throw new RuntimeException(e) - } + case e: IOException => throw new RuntimeException(e) } } @@ -45,8 +43,9 @@ object FixedPortTestUtils { enableControlledShutdown: Boolean = true, enableDeleteTopic: Boolean = false): Seq[Properties] = { val ports = FixedPortTestUtils.choosePorts(numConfigs) - (0 until numConfigs) - .map(node => TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node))) + (0 until numConfigs).map { node => + TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, ports(node)) + } } } diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index ee556d71d94da..dc30bf79fd436 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -45,17 +45,17 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { val consumers = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]() val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() - override def generateConfigs() = { + override def generateConfigs = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) cfgs.foreach(_.putAll(serverConfig)) cfgs.map(KafkaConfig.fromProps) } @Before override def setUp() { - val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) - val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) + val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) + val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, clientSaslProperties) super.setUp() producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) @@ -81,7 +81,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(producerConfig)) } @@ -89,7 +89,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { TestUtils.createNewConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, - saslProperties = this.saslProperties, + saslProperties = this.clientSaslProperties, props = Some(consumerConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala new file mode 100644 index 0000000000000..c63db6f1541e0 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/LogAppendTimeTest.scala @@ -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 kafka.api + +import java.util.Collections +import java.util.concurrent.TimeUnit + +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.record.TimestampType +import org.junit.{Before, Test} +import org.junit.Assert.{assertEquals, assertNotEquals, assertTrue} + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +/** + * Tests where the broker is configured to use LogAppendTime. For tests where LogAppendTime is configured via topic + * level configs, see the *ProducerSendTest classes. + */ +class LogAppendTimeTest extends IntegrationTestHarness { + val producerCount: Int = 1 + val consumerCount: Int = 1 + val serverCount: Int = 2 + + // This will be used for the offsets topic as well + serverConfig.put(KafkaConfig.LogMessageTimestampTypeProp, TimestampType.LOG_APPEND_TIME.name) + serverConfig.put(KafkaConfig.OffsetsTopicPartitionsProp, "1") + serverConfig.put(KafkaConfig.OffsetsTopicReplicationFactorProp, "2") + + private val topic = "topic" + + @Before + override def setUp() { + super.setUp() + TestUtils.createTopic(zkUtils, topic, servers = servers) + } + + @Test + def testProduceConsume() { + val producer = producers.head + val now = System.currentTimeMillis() + val createTime = now - TimeUnit.DAYS.toMillis(1) + val producerRecords = (1 to 10).map(i => new ProducerRecord(topic, null, createTime, s"key$i".getBytes, + s"value$i".getBytes)) + val recordMetadatas = producerRecords.map(producer.send).map(_.get(10, TimeUnit.SECONDS)) + recordMetadatas.foreach { recordMetadata => + assertTrue(recordMetadata.timestamp >= now) + assertTrue(recordMetadata.timestamp < now + TimeUnit.SECONDS.toMillis(60)) + } + + val consumer = consumers.head + consumer.subscribe(Collections.singleton(topic)) + val consumerRecords = new ArrayBuffer[ConsumerRecord[Array[Byte], Array[Byte]]] + TestUtils.waitUntilTrue(() => { + consumerRecords ++= consumer.poll(50).asScala + consumerRecords.size == producerRecords.size + }, s"Consumed ${consumerRecords.size} records until timeout instead of the expected ${producerRecords.size} records") + + consumerRecords.zipWithIndex.foreach { case (consumerRecord, index) => + val producerRecord = producerRecords(index) + val recordMetadata = recordMetadatas(index) + assertEquals(new String(producerRecord.key), new String(consumerRecord.key)) + assertEquals(new String(producerRecord.value), new String(consumerRecord.value)) + assertNotEquals(producerRecord.timestamp, consumerRecord.timestamp) + assertEquals(recordMetadata.timestamp, consumerRecord.timestamp) + assertEquals(TimestampType.LOG_APPEND_TIME, consumerRecord.timestampType) + } + } + +} diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index aefe5bd15ba25..50941cebeb9b2 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -12,7 +12,6 @@ */ package kafka.api - import java.util import java.util.regex.Pattern import java.util.{Collections, Locale, Properties} @@ -22,7 +21,7 @@ import kafka.server.KafkaConfig import kafka.utils.TestUtils import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{MetricName, TopicPartition} import org.apache.kafka.common.errors.InvalidTopicException import org.apache.kafka.common.record.{CompressionType, TimestampType} import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer, StringDeserializer, StringSerializer} @@ -528,7 +527,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString) val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) + saslProperties = clientSaslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) (0 until numRecords).foreach { i => producer.send(new ProducerRecord(tp.topic, tp.partition, i.toLong, s"key $i".getBytes, s"value $i".getBytes)) } @@ -848,28 +847,28 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor") producerProps.put("mock.interceptor.append", appendStr) - val testProducer = new KafkaProducer[String, String](producerProps, new StringSerializer, new StringSerializer) + val testProducer = new KafkaProducer(producerProps, new StringSerializer, new StringSerializer) // produce records val numRecords = 10 (0 until numRecords).map { i => - testProducer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key $i", s"value $i")) + testProducer.send(new ProducerRecord(tp.topic, tp.partition, s"key $i", s"value $i")) }.foreach(_.get) - assertEquals(numRecords, MockProducerInterceptor.ONSEND_COUNT.intValue()) - assertEquals(numRecords, MockProducerInterceptor.ON_SUCCESS_COUNT.intValue()) + assertEquals(numRecords, MockProducerInterceptor.ONSEND_COUNT.intValue) + assertEquals(numRecords, MockProducerInterceptor.ON_SUCCESS_COUNT.intValue) // send invalid record try { - testProducer.send(null, null) + testProducer.send(null) fail("Should not allow sending a null record") } catch { case _: Throwable => - assertEquals("Interceptor should be notified about exception", 1, MockProducerInterceptor.ON_ERROR_COUNT.intValue()) + assertEquals("Interceptor should be notified about exception", 1, MockProducerInterceptor.ON_ERROR_COUNT.intValue) assertEquals("Interceptor should not receive metadata with an exception when record is null", 0, MockProducerInterceptor.ON_ERROR_WITH_METADATA_COUNT.intValue()) } // create consumer with interceptor this.consumerConfig.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockConsumerInterceptor") - val testConsumer = new KafkaConsumer[String, String](this.consumerConfig, new StringDeserializer(), new StringDeserializer()) + val testConsumer = new KafkaConsumer(this.consumerConfig, new StringDeserializer, new StringDeserializer) testConsumer.assign(List(tp).asJava) testConsumer.seek(tp, 0) @@ -877,22 +876,22 @@ class PlaintextConsumerTest extends BaseConsumerTest { val records = consumeRecords(testConsumer, numRecords) for (i <- 0 until numRecords) { val record = records(i) - assertEquals(s"key $i", new String(record.key())) - assertEquals(s"value $i$appendStr".toUpperCase(Locale.ROOT), new String(record.value())) + assertEquals(s"key $i", new String(record.key)) + assertEquals(s"value $i$appendStr".toUpperCase(Locale.ROOT), new String(record.value)) } // commit sync and verify onCommit is called - val commitCountBefore = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue() + val commitCountBefore = MockConsumerInterceptor.ON_COMMIT_COUNT.intValue testConsumer.commitSync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(2L))).asJava) assertEquals(2, testConsumer.committed(tp).offset) - assertEquals(commitCountBefore + 1, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue()) + assertEquals(commitCountBefore + 1, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue) // commit async and verify onCommit is called val commitCallback = new CountConsumerCommitCallback() testConsumer.commitAsync(Map[TopicPartition, OffsetAndMetadata]((tp, new OffsetAndMetadata(5L))).asJava, commitCallback) awaitCommitCallback(testConsumer, commitCallback) assertEquals(5, testConsumer.committed(tp).offset) - assertEquals(commitCountBefore + 2, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue()) + assertEquals(commitCountBefore + 2, MockConsumerInterceptor.ON_COMMIT_COUNT.intValue) testConsumer.close() testProducer.close() @@ -1230,6 +1229,105 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(500, consumer0.committed(tp2).offset) } + + @Test + def testPerPartitionLagMetricsCleanUpWithSubscribe() { + val numMessages = 1000 + val topic2 = "topic2" + TestUtils.createTopic(this.zkUtils, topic2, 2, serverCount, this.servers) + // send some messages. + sendRecords(numMessages, tp) + // Test subscribe + // Create a consumer and consumer some messages. + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithSubscribe") + consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithSubscribe") + val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + try { + val listener0 = new TestConsumerReassignmentListener + consumer.subscribe(List(topic, topic2).asJava, listener0) + var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() + TestUtils.waitUntilTrue(() => { + records = consumer.poll(100) + !records.records(tp).isEmpty + }, "Consumer did not consume any message before timeout.") + assertEquals("should be assigned once", 1, listener0.callsToAssigned) + // Verify the metric exist. + val tags = Collections.singletonMap("client-id", "testPerPartitionLagMetricsCleanUpWithSubscribe") + val fetchLag0 = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) + assertNotNull(fetchLag0) + val expectedLag = numMessages - records.count + assertEquals(s"The lag should be $expectedLag", expectedLag, fetchLag0.value, epsilon) + + // Remove topic from subscription + consumer.subscribe(List(topic2).asJava, listener0) + TestUtils.waitUntilTrue(() => { + consumer.poll(100) + listener0.callsToAssigned >= 2 + }, "Expected rebalance did not occur.") + // Verify the metric has gone + assertNull(consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) + assertNull(consumer.metrics.get(new MetricName(tp2 + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) + } finally { + consumer.close() + } + } + + @Test + def testPerPartitionLagMetricsCleanUpWithAssign() { + val numMessages = 1000 + // Test assign + // send some messages. + sendRecords(numMessages, tp) + sendRecords(numMessages, tp2) + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithAssign") + consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagMetricsCleanUpWithAssign") + val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + try { + consumer.assign(List(tp).asJava) + var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() + TestUtils.waitUntilTrue(() => { + records = consumer.poll(100) + !records.records(tp).isEmpty + }, "Consumer did not consume any message before timeout.") + // Verify the metric exist. + val tags = Collections.singletonMap("client-id", "testPerPartitionLagMetricsCleanUpWithAssign") + val fetchLag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) + assertNotNull(fetchLag) + val expectedLag = numMessages - records.count + assertEquals(s"The lag should be $expectedLag", expectedLag, fetchLag.value, epsilon) + + consumer.assign(List(tp2).asJava) + TestUtils.waitUntilTrue(() => !consumer.poll(100).isEmpty, "Consumer did not consume any message before timeout.") + assertNull(consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags))) + } finally { + consumer.close() + } + } + + @Test + def testPerPartitionLagWithMaxPollRecords() { + val numMessages = 1000 + val maxPollRecords = 10 + sendRecords(numMessages, tp) + consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testPerPartitionLagWithMaxPollRecords") + consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString) + val consumer = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer()) + consumer.assign(List(tp).asJava) + try { + var records: ConsumerRecords[Array[Byte], Array[Byte]] = ConsumerRecords.empty() + TestUtils.waitUntilTrue(() => { + records = consumer.poll(100) + !records.isEmpty + }, "Consumer did not consume any message before timeout.") + val tags = Collections.singletonMap("client-id", "testPerPartitionLagWithMaxPollRecords") + val lag = consumer.metrics.get(new MetricName(tp + ".records-lag", "consumer-fetch-manager-metrics", "", tags)) + assertEquals(s"The lag should be ${numMessages - records.count}", numMessages - records.count, lag.value, epsilon) + } finally { + consumer.close() + } + } + def runMultiConsumerSessionTimeoutTest(closeConsumer: Boolean): Unit = { // use consumers defined in this class plus one additional consumer // Use topic defined in this class + one additional topic diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index a75e7c73b1cee..10063a99801d3 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -22,53 +22,31 @@ import java.util.concurrent.ExecutionException import kafka.log.LogConfig import kafka.utils.TestUtils - import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} -import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.errors.{InvalidTimestampException, SerializationException} import org.apache.kafka.common.record.TimestampType -import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.Assert._ import org.junit.Test class PlaintextProducerSendTest extends BaseProducerSendTest { - @Test - def testSerializerConstructors() { - try { - createNewProducerWithNoSerializer(brokerList) - fail("Instantiating a producer without specifying a serializer should cause a ConfigException") - } catch { - case _ : ConfigException => // this is ok - } - - // create a producer with explicit serializers should succeed - createNewProducerWithExplicitSerializer(brokerList) - } - - private def createNewProducerWithNoSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = { - val producerProps = new Properties() - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - registerProducer(new KafkaProducer(producerProps)) - } - - private def createNewProducerWithExplicitSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = { + @Test(expected = classOf[SerializationException]) + def testWrongSerializer() { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - registerProducer(new KafkaProducer(producerProps, new ByteArraySerializer, new ByteArraySerializer)) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + val producer = registerProducer(new KafkaProducer(producerProps)) + val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + producer.send(record) } @Test - def testWrongSerializer() { - // send a record with a wrong type should receive a serialization exception - try { - val producer = createProducerWithWrongSerializer(brokerList) - val record5 = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) - producer.send(record5) - fail("Should have gotten a SerializationException") - } catch { - case _: SerializationException => // this is ok - } + def testBatchSizeZero() { + val producerProps = new Properties() + producerProps.setProperty(ProducerConfig.BATCH_SIZE_CONFIG, "0") + val producer = createProducer(brokerList = brokerList, lingerMs = Long.MaxValue, props = Some(producerProps)) + sendAndVerify(producer) } @Test @@ -96,7 +74,7 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { try { // Send a message to auto-create the topic - val record = new ProducerRecord[Array[Byte], Array[Byte]](topic, null, "key".getBytes, "value".getBytes) + val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) assertEquals("Should have offset 0", 0L, producer.send(record).get.offset) // double check that the topic is created with leader elected @@ -137,12 +115,4 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { } } - private def createProducerWithWrongSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = { - val producerProps = new Properties() - producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") - registerProducer(new KafkaProducer(producerProps)) - } - } diff --git a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala index 992649a0f9eff..826eb5ce8babf 100644 --- a/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslEndToEndAuthorizationTest.scala @@ -16,10 +16,9 @@ */ package kafka.api -import java.io.File import java.util.Properties -import kafka.utils.{JaasTestUtils,TestUtils} +import kafka.utils.TestUtils import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.config.SaslConfigs import org.apache.kafka.common.errors.GroupAuthorizationException @@ -30,24 +29,24 @@ import scala.collection.JavaConverters._ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { override protected def securityProtocol = SecurityProtocol.SASL_SSL - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) protected def kafkaClientSaslMechanism: String protected def kafkaServerSaslMechanisms: List[String] @Before override def setUp { - startSasl(Both, List(kafkaClientSaslMechanism), kafkaServerSaslMechanisms) + startSasl(Both, kafkaServerSaslMechanisms, Option(kafkaClientSaslMechanism)) super.setUp } // Use JAAS configuration properties for clients so that dynamic JAAS configuration is also tested by this set of tests - override protected def setJaasConfiguration(mode: SaslSetupMode, serverMechanisms: List[String], clientMechanisms: List[String], - serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) { + override protected def setJaasConfiguration(mode: SaslSetupMode, serverMechanisms: List[String], clientMechanism: Option[String]) { // create static config with client login context with credentials for JaasTestUtils 'client2' - super.setJaasConfiguration(mode, kafkaServerSaslMechanisms, clientMechanisms, serverKeytabFile, clientKeytabFile) + super.setJaasConfiguration(mode, kafkaServerSaslMechanisms, clientMechanism) // set dynamic properties with credentials for JaasTestUtils 'client1' - val clientLoginContext = JaasTestUtils.clientLoginModule(kafkaClientSaslMechanism, clientKeytabFile) + val clientLoginContext = jaasClientLoginModule(kafkaClientSaslMechanism) producerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext) consumerConfig.put(SaslConfigs.SASL_JAAS_CONFIG, clientLoginContext) } @@ -70,7 +69,7 @@ abstract class SaslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { val consumer2 = TestUtils.createNewConsumer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - saslProperties = saslProperties, + saslProperties = clientSaslProperties, props = Some(consumer2Config)) consumers += consumer2 diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index 5814e9463560c..3ff133f228ac4 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -23,11 +23,11 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne override protected val zkSaslEnabled = true override protected val kafkaClientSaslMechanism = "PLAIN" override protected val kafkaServerSaslMechanisms = List("GSSAPI", "PLAIN") - override protected def allKafkaClientSaslMechanisms = List("PLAIN", "GSSAPI") this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) @Test def testMultipleBrokerMechanisms() { @@ -35,7 +35,7 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne val plainSaslProducer = producers.head val plainSaslConsumer = consumers.head - val gssapiSaslProperties = kafkaSaslProperties("GSSAPI") + val gssapiSaslProperties = kafkaClientSaslProperties("GSSAPI", dynamicJaasConfig = true) val gssapiSaslProducer = TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala index bdca577c0e878..125d4318ebe8b 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala @@ -23,5 +23,6 @@ class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarne this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) } diff --git a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala index fe0204ae51932..86db4074f9476 100644 --- a/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslScramSslEndToEndAuthorizationTest.scala @@ -32,7 +32,7 @@ class SaslScramSslEndToEndAuthorizationTest extends SaslEndToEndAuthorizationTes override def configureSecurityBeforeServersStart() { super.configureSecurityBeforeServersStart() - zkUtils.makeSurePersistentPathExists(ZkUtils.EntityConfigChangesPath) + zkUtils.makeSurePersistentPathExists(ZkUtils.ConfigChangesPath) def configCommandArgs(username: String, password: String) : Array[String] = { val credentials = kafkaServerSaslMechanisms.map(m => s"$m=[iterations=4096,password=$password]") diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index c1e2da2cbf797..36b9d41e0584d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -43,32 +43,36 @@ trait SaslSetup { private val workDir = TestUtils.tempDir() private val kdcConf = MiniKdc.createConfig private var kdc: MiniKdc = null + private var serverKeytabFile: Option[File] = null + private var clientKeytabFile: Option[File] = null - def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String]) { + def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String]) { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() - val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanisms.contains("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI")) + val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanism == Some("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI")) if (hasKerberos) { val serverKeytabFile = TestUtils.tempFile() val clientKeytabFile = TestUtils.tempFile() - setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, Some(serverKeytabFile), Some(clientKeytabFile)) + this.clientKeytabFile = Some(clientKeytabFile) + this.serverKeytabFile = Some(serverKeytabFile) kdc = new MiniKdc(kdcConf, workDir) kdc.start() kdc.createPrincipal(serverKeytabFile, JaasTestUtils.KafkaServerPrincipalUnqualifiedName + "/localhost") kdc.createPrincipal(clientKeytabFile, JaasTestUtils.KafkaClientPrincipalUnqualifiedName, JaasTestUtils.KafkaClientPrincipalUnqualifiedName2) } else { - setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms) + this.clientKeytabFile = None + this.serverKeytabFile = None } + setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanism) if (mode == Both || mode == ZkSasl) System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") } - protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], - serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) { + protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String]) { val jaasFile = mode match { case ZkSasl => JaasTestUtils.writeZkFile() - case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) - case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) + case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanism, serverKeytabFile, clientKeytabFile) + case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanism, serverKeytabFile, clientKeytabFile) } // This will cause a reload of the Configuration singleton when `getConfiguration` is called Configuration.setConfiguration(null) @@ -85,13 +89,20 @@ trait SaslSetup { Configuration.setConfiguration(null) } - def kafkaSaslProperties(clientSaslMechanism: String, serverSaslMechanisms: Option[Seq[String]] = None) = { + def kafkaServerSaslProperties(serverSaslMechanisms: Seq[String], interBrokerSaslMechanism: String) = { + val props = new Properties + props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, interBrokerSaslMechanism) + props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverSaslMechanisms.mkString(",")) + props + } + + def kafkaClientSaslProperties(clientSaslMechanism: String, dynamicJaasConfig: Boolean = false) = { val props = new Properties props.put(SaslConfigs.SASL_MECHANISM, clientSaslMechanism) - serverSaslMechanisms.foreach { serverMechanisms => - props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, clientSaslMechanism) - props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms.mkString(",")) - } + if (dynamicJaasConfig) + props.put(SaslConfigs.SASL_JAAS_CONFIG, jaasClientLoginModule(clientSaslMechanism)) props } + + def jaasClientLoginModule(clientSaslMechanism: String): String = JaasTestUtils.clientLoginModule(clientSaslMechanism, clientKeytabFile) } diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 8fd3eb4422e0e..97faa36e7fa2c 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -20,16 +20,12 @@ trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { protected val kafkaClientSaslMechanism = "GSSAPI" protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism) - // Override this list to enable client login modules for multiple mechanisms for testing - // of multi-mechanism brokers with clients using different mechanisms in a single JVM - protected def allKafkaClientSaslMechanisms = List(kafkaClientSaslMechanism) - @Before override def setUp() { if (zkSaslEnabled) - startSasl(Both, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) + startSasl(Both, kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism)) else - startSasl(KafkaSasl, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) + startSasl(KafkaSasl, kafkaServerSaslMechanisms, Some(kafkaClientSaslMechanism)) super.setUp } diff --git a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala index 365c0ba5b5886..064e783508130 100644 --- a/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/SslEndToEndAuthorizationTest.scala @@ -29,7 +29,7 @@ class SslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { @Before override def setUp { - startSasl(ZkSasl, List.empty, List.empty) + startSasl(ZkSasl, List.empty, None) super.setUp } } diff --git a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala index 4677c8ce9ab2b..c8d0a77aac146 100644 --- a/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala +++ b/core/src/test/scala/integration/kafka/api/UserQuotaTest.scala @@ -29,7 +29,8 @@ class UserQuotaTest extends BaseQuotaTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) override protected val zkSaslEnabled = false - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) override val userPrincipal = JaasTestUtils.KafkaClientPrincipalUnqualifiedName2 override val producerQuotaId = QuotaId(Some(userPrincipal), None) diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index f9eb61c00aad7..aa3dbc4adc528 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -20,6 +20,7 @@ import kafka.server.DynamicConfig.Broker._ import kafka.server.KafkaConfig._ import org.apache.kafka.common.errors.{InvalidReplicaAssignmentException, InvalidReplicationFactorException, InvalidTopicException, TopicExistsException} import org.apache.kafka.common.metrics.Quota +import org.easymock.EasyMock import org.junit.Assert._ import org.junit.Test import java.util.Properties @@ -159,6 +160,21 @@ class AdminTest extends ZooKeeperTestHarness with Logging with RackAwareTest { } } + @Test + def testConcurrentTopicCreation() { + val topic = "test.topic" + + // simulate the ZK interactions that can happen when a topic is concurrently created by multiple processes + val zkMock = EasyMock.createNiceMock(classOf[ZkUtils]) + EasyMock.expect(zkMock.pathExists(s"/brokers/topics/$topic")).andReturn(false) + EasyMock.expect(zkMock.getAllTopics).andReturn(Seq("some.topic", topic, "some.other.topic")) + EasyMock.replay(zkMock) + + intercept[TopicExistsException] { + AdminUtils.validateCreateOrUpdateTopic(zkMock, topic, Map.empty, new Properties, update = false) + } + } + private def getBrokersWithPartitionDir(servers: Iterable[KafkaServer], topic: String, partitionId: Int): Set[Int] = { servers.filter(server => new File(server.config.logDirs.head, topic + "-" + partitionId).exists) .map(_.config.brokerId) diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala index 95ee5d386ccae..c211c245eb18c 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala @@ -16,12 +16,13 @@ import kafka.common.{AdminCommandFailedException, TopicAndPartition} import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.TestUtils._ import kafka.utils.ZkUtils._ -import kafka.utils.{CoreUtils, Logging, ZkUtils} +import kafka.utils.{CoreUtils, Logging, TestUtils, ZkUtils} import kafka.zk.ZooKeeperTestHarness import org.junit.Assert.{assertEquals, assertTrue} import org.junit.{After, Before, Test} import kafka.admin.ReplicationQuotaUtils._ -import scala.collection.Seq + +import scala.collection.{Map, Seq} class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { @@ -58,7 +59,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { waitForReassignmentToComplete() //Then the replica should be on 101 - assertEquals(zkUtils.getPartitionAssignmentForTopics(Seq(topicName)).get(topicName).get(partition), Seq(101)) + assertEquals(Seq(101), zkUtils.getPartitionAssignmentForTopics(Seq(topicName)).get(topicName).get(partition)) } @Test @@ -79,7 +80,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { //Then the replicas should span all three brokers val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) - assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101, 102)) + assertEquals(Seq(100, 101, 102), actual.values.flatten.toSeq.distinct.sorted) } @Test @@ -100,7 +101,43 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { //Then replicas should only span the first two brokers val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) - assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101)) + assertEquals(Seq(100, 101), actual.values.flatten.toSeq.distinct.sorted) + } + + @Test + def shouldMoveSubsetOfPartitions() { + //Given partitions on 3 of 3 brokers + val brokers = Array(100, 101, 102) + startBrokers(brokers) + createTopic(zkUtils, "topic1", Map( + 0 -> Seq(100, 101), + 1 -> Seq(101, 102), + 2 -> Seq(102, 100) + ), servers = servers) + createTopic(zkUtils, "topic2", Map( + 0 -> Seq(100, 101), + 1 -> Seq(101, 102), + 2 -> Seq(102, 100) + ), servers = servers) + + val proposed: Map[TopicAndPartition, Seq[Int]] = Map( + TopicAndPartition("topic1", 0) -> Seq(100, 102), + TopicAndPartition("topic1", 2) -> Seq(100, 102), + TopicAndPartition("topic2", 2) -> Seq(100, 102) + ) + + //When rebalancing + ReassignPartitionsCommand.executeAssignment(zkUtils, ZkUtils.formatAsReassignmentJson(proposed)) + waitForReassignmentToComplete() + + //Then the proposed changes should have been made + val actual = zkUtils.getPartitionAssignmentForTopics(Seq("topic1", "topic2")) + assertEquals(Seq(100, 102), actual("topic1")(0))//changed + assertEquals(Seq(101, 102), actual("topic1")(1)) + assertEquals(Seq(100, 102), actual("topic1")(2))//changed + assertEquals(Seq(100, 101), actual("topic2")(0)) + assertEquals(Seq(101, 102), actual("topic2")(1)) + assertEquals(Seq(100, 102), actual("topic2")(2))//changed } @Test @@ -135,7 +172,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { //Check move occurred val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) - assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(101, 102)) + assertEquals(Seq(101, 102), actual.values.flatten.toSeq.distinct.sorted) //Then command should have taken longer than the throttle rate assertTrue(s"Expected replication to be > ${expectedDurationSecs * 0.9 * 1000} but was $took", took > expectedDurationSecs * 0.9 * 1000) @@ -233,7 +270,7 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { //Check move occurred val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) - assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(101, 102)) + assertEquals(Seq(101, 102), actual.values.flatten.toSeq.distinct.sorted) } @Test(expected = classOf[AdminCommandFailedException]) @@ -246,6 +283,46 @@ class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { ReassignPartitionsCommand.executeAssignment(zkUtils, s"""{"version":1,"partitions":[{"topic":"$topicName","partition":1,"replicas":[101]}]}""") } + @Test + def shouldPerformThrottledReassignmentOverVariousTopics() { + val throttle = 1000L + + //Given four brokers + servers = TestUtils.createBrokerConfigs(4, zkConnect, false).map(conf => TestUtils.createServer(KafkaConfig.fromProps(conf))) + + //With up several small topics + createTopic(zkUtils, "orders", Map(0 -> List(0, 1, 2), 1 -> List(0, 1, 2)), servers) + createTopic(zkUtils, "payments", Map(0 -> List(0, 1), 1 -> List(0, 1)), servers) + createTopic(zkUtils, "deliveries", Map(0 -> List(0)), servers) + createTopic(zkUtils, "customers", Map(0 -> List(0), 1 -> List(1), 2 -> List(2), 3 -> List(3)), servers) + + //Define a move for some of them + val move = Map( + TopicAndPartition("orders", 0) -> Seq(0, 2, 3),//moves + TopicAndPartition("orders", 1) -> Seq(0, 1, 2),//stays + TopicAndPartition("payments", 1) -> Seq(1, 2), //only define one partition as moving + TopicAndPartition("deliveries", 0) -> Seq(1, 2) //increase replication factor + ) + + //When we run a throttled reassignment + new ReassignPartitionsCommand(zkUtils, move).reassignPartitions(throttle) + + waitForReassignmentToComplete() + + //Check moved replicas did move + assertEquals(Seq(0, 2, 3), zkUtils.getReplicasForPartition("orders", 0)) + assertEquals(Seq(0, 1, 2), zkUtils.getReplicasForPartition("orders", 1)) + assertEquals(Seq(1, 2), zkUtils.getReplicasForPartition("payments", 1)) + assertEquals(Seq(1, 2), zkUtils.getReplicasForPartition("deliveries", 0)) + + //Check untouched replicas are still there + assertEquals(Seq(0, 1), zkUtils.getReplicasForPartition("payments", 0)) + assertEquals(Seq(0), zkUtils.getReplicasForPartition("customers", 0)) + assertEquals(Seq(1), zkUtils.getReplicasForPartition("customers", 1)) + assertEquals(Seq(2), zkUtils.getReplicasForPartition("customers", 2)) + assertEquals(Seq(3), zkUtils.getReplicasForPartition("customers", 3)) + } + def waitForReassignmentToComplete() { waitUntilTrue(() => !zkUtils.pathExists(ReassignPartitionsPath), s"Znode ${ZkUtils.ReassignPartitionsPath} wasn't deleted") } diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala index 924daf8676749..5ecc19bde25be 100644 --- a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala @@ -29,6 +29,7 @@ import org.easymock.EasyMock._ import org.easymock.{Capture, CaptureType, EasyMock} import org.junit.{Before, Test} import org.junit.Assert.{assertEquals, assertNull, fail} + import scala.collection.{Seq, mutable} import scala.collection.JavaConversions._ @@ -57,6 +58,38 @@ class ReassignPartitionsCommandTest extends Logging { assertEquals(1, calls) } + @Test + def shouldFindMovingReplicasWhenProposedIsSubsetOfExisting() { + val assigner = new ReassignPartitionsCommand(null, null) + + //Given we have more existing partitions than we are proposing + val existingSuperset = Map( + TopicAndPartition("topic1", 0) -> Seq(100, 101), + TopicAndPartition("topic1", 1) -> Seq(100, 102), + TopicAndPartition("topic1", 2) -> Seq(100, 101), + TopicAndPartition("topic2", 0) -> Seq(100, 101, 102), + TopicAndPartition("topic3", 0) -> Seq(100, 101, 102) + ) + val proposedSubset = Map( + TopicAndPartition("topic1", 0) -> Seq(101, 102), + TopicAndPartition("topic1", 1) -> Seq(102), + TopicAndPartition("topic1", 2) -> Seq(100, 101, 102) + ) + + val mock = new TestAdminUtils { + override def changeTopicConfig(zkUtils: ZkUtils, topic: String, configChange: Properties): Unit = { + assertEquals("0:102,2:102", configChange.get(FollowerReplicationThrottledReplicasProp)) + assertEquals("0:100,0:101,2:100,2:101", configChange.get(LeaderReplicationThrottledReplicasProp)) + assertEquals("topic1", topic) + calls += 1 + } + } + + //Then replicas should assign correctly (based on the proposed map) + assigner.assignThrottledReplicas(existingSuperset, proposedSubset, mock) + assertEquals(1, calls) + } + @Test def shouldFindMovingReplicasMultiplePartitions() { val control = TopicAndPartition("topic1", 2) -> Seq(100, 102) diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index 8ce7c90436f44..52158673e435e 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -49,7 +49,7 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging with RackAwareT assertTrue("Properties after creation have incorrect value", props.getProperty(cleanupKey).equals(cleanupVal)) // pre-create the topic config changes path to avoid a NoNodeException - zkUtils.createPersistentPath(EntityConfigChangesPath) + zkUtils.createPersistentPath(ConfigChangesPath) // modify the topic to add new partitions val numPartitionsModified = 3 diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala index 20e512faaaaff..f394369a76142 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala @@ -25,7 +25,7 @@ import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager} import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest} +import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest, OffsetFetchResponse} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.easymock.{Capture, EasyMock, IAnswer} import org.junit.{After, Before, Test} @@ -305,8 +305,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.reset(replicaManager) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andReturn(None) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) timer.advanceClock(DefaultSessionTimeout + 100) @@ -748,6 +747,62 @@ class GroupCoordinatorResponseTest extends JUnitSuite { assertEquals(Errors.NONE.code, commitOffsetResult(tp)) } + @Test + def testFetchOffsets() { + val tp = new TopicPartition("topic", 0) + val offset = OffsetAndMetadata(0) + + val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp -> offset)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp)) + + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp))) + assertEquals(Errors.NONE, error) + assertEquals(Some(0), partitionData.get(tp).map(_.offset)) + } + + @Test + def testFetchOffsetForUnknownPartition(): Unit = { + val tp = new TopicPartition("topic", 0) + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId, Some(Seq(tp))) + assertEquals(Errors.NONE, error) + assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), partitionData.get(tp).map(_.offset)) + } + + @Test + def testFetchOffsetNotCoordinatorForGroup(): Unit = { + val tp = new TopicPartition("topic", 0) + val (error, partitionData) = groupCoordinator.handleFetchOffsets(otherGroupId, Some(Seq(tp))) + assertEquals(Errors.NOT_COORDINATOR_FOR_GROUP, error) + assertTrue(partitionData.isEmpty) + } + + @Test + def testFetchAllOffsets() { + val tp1 = new TopicPartition("topic", 0) + val tp2 = new TopicPartition("topic", 1) + val tp3 = new TopicPartition("other-topic", 0) + val offset1 = OffsetAndMetadata(15) + val offset2 = OffsetAndMetadata(16) + val offset3 = OffsetAndMetadata(17) + + assertEquals((Errors.NONE, Map.empty), groupCoordinator.handleFetchOffsets(groupId)) + + val commitOffsetResult = commitOffsets(groupId, OffsetCommitRequest.DEFAULT_MEMBER_ID, + OffsetCommitRequest.DEFAULT_GENERATION_ID, immutable.Map(tp1 -> offset1, tp2 -> offset2, tp3 -> offset3)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp1)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp2)) + assertEquals(Errors.NONE.code, commitOffsetResult(tp3)) + + val (error, partitionData) = groupCoordinator.handleFetchOffsets(groupId) + assertEquals(Errors.NONE, error) + assertEquals(3, partitionData.size) + assertTrue(partitionData.forall(_._2.error == Errors.NONE)) + assertEquals(Some(offset1.offset), partitionData.get(tp1).map(_.offset)) + assertEquals(Some(offset2.offset), partitionData.get(tp2).map(_.offset)) + assertEquals(Some(offset3.offset), partitionData.get(tp3).map(_.offset)) + } + @Test def testCommitOffsetInAwaitingSync() { val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID @@ -996,11 +1051,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { override def answer = capturedArgument.getValue.apply( Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> - new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP) + new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback) @@ -1078,11 +1132,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite { EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { override def answer = capturedArgument.getValue.apply( Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> - new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP) + new PartitionResponse(Errors.NONE, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback) @@ -1093,8 +1146,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite { val (responseFuture, responseCallback) = setupHeartbeatCallback EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andReturn(None) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes() + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(Some(Record.MAGIC_VALUE_V1)).anyTimes() EasyMock.replay(replicaManager) groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback) diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala index e3dca329529ff..a4c3448c4ac03 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala @@ -17,20 +17,23 @@ package kafka.coordinator +import java.nio.ByteBuffer + import kafka.api.ApiVersion import kafka.cluster.Partition import kafka.common.{OffsetAndMetadata, Topic} -import kafka.log.LogAppendInfo -import kafka.server.{KafkaConfig, ReplicaManager} +import kafka.log.{Log, LogAppendInfo} +import kafka.server.{FetchDataInfo, KafkaConfig, LogOffsetMetadata, ReplicaManager} import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils} import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors -import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType} +import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record, TimestampType} import org.apache.kafka.common.requests.OffsetFetchResponse import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.easymock.{Capture, EasyMock, IAnswer} -import org.junit.{After, Before, Test} -import org.junit.Assert._ +import org.junit.{Before, Test} +import org.junit.Assert.{assertEquals, assertFalse, assertTrue} +import kafka.utils.TestUtils.fail import scala.collection._ import JavaConverters._ @@ -79,10 +82,153 @@ class GroupMetadataManagerTest { partition = EasyMock.niceMock(classOf[Partition]) } - @After - def tearDown() { - EasyMock.reset(replicaManager) - EasyMock.reset(partition) + @Test + def testLoadOffsetsWithoutGroup() { + val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) + val startOffset = 15L + + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + new TopicPartition("foo", 1) -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val records = MemoryRecords.withRecords(startOffset, offsetCommitRecords: _*) + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache")) + assertEquals(groupId, group.groupId) + assertEquals(Empty, group.currentState) + assertEquals(committedOffsets.size, group.allOffsets.size) + committedOffsets.foreach { case (topicPartition, offset) => + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } + } + + @Test + def testLoadOffsetsWithTombstones() { + val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) + val startOffset = 15L + + val tombstonePartition = new TopicPartition("foo", 1) + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + tombstonePartition -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val tombstone = Record.create(GroupMetadataManager.offsetCommitKey(groupId, tombstonePartition), null) + val records = MemoryRecords.withRecords(startOffset, offsetCommitRecords ++ Seq(tombstone): _*) + + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache")) + assertEquals(groupId, group.groupId) + assertEquals(Empty, group.currentState) + assertEquals(committedOffsets.size - 1, group.allOffsets.size) + committedOffsets.foreach { case (topicPartition, offset) => + if (topicPartition == tombstonePartition) + assertEquals(None, group.offset(topicPartition)) + else + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } + } + + @Test + def testLoadOffsetsAndGroup() { + val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) + val startOffset = 15L + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + new TopicPartition("foo", 1) -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val memberId = "98098230493" + val groupMetadataRecord = buildStableGroupRecordWithMember(memberId) + val records = MemoryRecords.withRecords(startOffset, offsetCommitRecords ++ Seq(groupMetadataRecord): _*) + + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + val group = groupMetadataManager.getGroup(groupId).getOrElse(fail("Group was not loaded into the cache")) + assertEquals(groupId, group.groupId) + assertEquals(Stable, group.currentState) + assertEquals(memberId, group.leaderId) + assertEquals(Set(memberId), group.allMembers) + assertEquals(committedOffsets.size, group.allOffsets.size) + committedOffsets.foreach { case (topicPartition, offset) => + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } + } + + @Test + def testLoadGroupWithTombstone() { + val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) + val startOffset = 15L + + val memberId = "98098230493" + val groupMetadataRecord = buildStableGroupRecordWithMember(memberId) + val groupMetadataTombstone = Record.create(GroupMetadataManager.groupMetadataKey(groupId), null) + val records = MemoryRecords.withRecords(startOffset, Seq(groupMetadataRecord, groupMetadataTombstone): _*) + + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + assertEquals(None, groupMetadataManager.getGroup(groupId)) + } + + @Test + def testOffsetWriteAfterGroupRemoved(): Unit = { + // this test case checks the following scenario: + // 1. the group exists at some point in time, but is later removed (because all members left) + // 2. a "simple" consumer (i.e. not a consumer group) then uses the same groupId to commit some offsets + + val groupMetadataTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) + val startOffset = 15L + + val committedOffsets = Map( + new TopicPartition("foo", 0) -> 23L, + new TopicPartition("foo", 1) -> 455L, + new TopicPartition("bar", 0) -> 8992L + ) + val offsetCommitRecords = createCommittedOffsetRecords(committedOffsets) + val memberId = "98098230493" + val groupMetadataRecord = buildStableGroupRecordWithMember(memberId) + val groupMetadataTombstone = Record.create(GroupMetadataManager.groupMetadataKey(groupId), null) + val records = MemoryRecords.withRecords(startOffset, + Seq(groupMetadataRecord, groupMetadataTombstone) ++ offsetCommitRecords: _*) + + expectGroupMetadataLoad(groupMetadataTopicPartition, startOffset, records) + + EasyMock.replay(replicaManager) + + groupMetadataManager.loadGroupsAndOffsets(groupMetadataTopicPartition, _ => ()) + + val group = groupMetadataManager.getGroup(groupId).getOrElse(TestUtils.fail("Group was not loaded into the cache")) + assertEquals(groupId, group.groupId) + assertEquals(Empty, group.currentState) + assertEquals(committedOffsets.size, group.allOffsets.size) + committedOffsets.foreach { case (topicPartition, offset) => + assertEquals(Some(offset), group.offset(topicPartition).map(_.offset)) + } } @Test @@ -156,7 +302,7 @@ class GroupMetadataManagerTest { val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List(("protocol", Array[Byte]()))) member.awaitingJoinCallback = _ => () - group.add(memberId, member) + group.add(member) group.transitionTo(PreparingRebalance) group.initNextGeneration() @@ -175,7 +321,7 @@ class GroupMetadataManagerTest { @Test def testStoreNonEmptyGroupWhenCoordinatorHasMoved() { - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(None) val memberId = "memberId" val clientId = "clientId" val clientHost = "localhost" @@ -185,7 +331,7 @@ class GroupMetadataManagerTest { val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List(("protocol", Array[Byte]()))) member.awaitingJoinCallback = _ => () - group.add(memberId, member) + group.add(member) group.transitionTo(PreparingRebalance) group.initNextGeneration() @@ -244,7 +390,7 @@ class GroupMetadataManagerTest { @Test def testCommitOffsetWhenCoordinatorHasMoved() { - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andReturn(None) val memberId = "" val generationId = -1 val topicPartition = new TopicPartition("foo", 0) @@ -264,7 +410,7 @@ class GroupMetadataManagerTest { commitErrors = Some(errors) } - val delayedStoreOpt = groupMetadataManager.prepareStoreOffsets(group, memberId, generationId, offsets, callback) + groupMetadataManager.prepareStoreOffsets(group, memberId, generationId, offsets, callback) assertFalse(commitErrors.isEmpty) val maybeError = commitErrors.get.get(topicPartition) @@ -392,8 +538,7 @@ class GroupMetadataManagerTest { EasyMock.reset(partition) val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture() - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition)) EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt())) .andReturn(LogAppendInfo.UnknownLogAppendInfo) @@ -438,8 +583,7 @@ class GroupMetadataManagerTest { EasyMock.reset(partition) val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture() - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.LOG_APPEND_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) EasyMock.expect(replicaManager.getPartition(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId))).andStubReturn(Some(partition)) EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt())) .andReturn(LogAppendInfo.UnknownLogAppendInfo) @@ -456,7 +600,8 @@ class GroupMetadataManagerTest { assertTrue(metadataTombstone.hasKey) assertTrue(metadataTombstone.hasNullValue) assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic) - assertEquals(TimestampType.LOG_APPEND_TIME, metadataTombstone.timestampType) + // Use CREATE_TIME, like the producer. The conversion to LOG_APPEND_TIME (if necessary) happens automatically. + assertEquals(TimestampType.CREATE_TIME, metadataTombstone.timestampType) assertTrue(metadataTombstone.timestamp > 0) val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey] @@ -557,7 +702,7 @@ class GroupMetadataManagerTest { val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout, protocolType, List(("protocol", Array[Byte]()))) member.awaitingJoinCallback = _ => () - group.add(memberId, member) + group.add(member) group.transitionTo(PreparingRebalance) group.initNextGeneration() @@ -613,11 +758,53 @@ class GroupMetadataManagerTest { EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] { override def answer = capturedArgument.getValue.apply( Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) -> - new PartitionResponse(error.code, 0L, Record.NO_TIMESTAMP) + new PartitionResponse(error, 0L, Record.NO_TIMESTAMP) ) )}) - EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())) - .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)) + EasyMock.expect(replicaManager.getMagic(EasyMock.anyObject())).andStubReturn(Some(Record.MAGIC_VALUE_V1)) + } + + private def buildStableGroupRecordWithMember(memberId: String): Record = { + val group = new GroupMetadata(groupId) + group.transitionTo(PreparingRebalance) + val memberProtocols = List(("roundrobin", Array.emptyByteArray)) + val member = new MemberMetadata(memberId, groupId, "clientId", "clientHost", 30000, 10000, "consumer", memberProtocols) + group.add(member) + member.awaitingJoinCallback = _ => {} + group.initNextGeneration() + group.transitionTo(Stable) + + val groupMetadataKey = GroupMetadataManager.groupMetadataKey(groupId) + val groupMetadataValue = GroupMetadataManager.groupMetadataValue(group, Map(memberId -> Array.empty[Byte])) + Record.create(groupMetadataKey, groupMetadataValue) + } + + private def expectGroupMetadataLoad(groupMetadataTopicPartition: TopicPartition, + startOffset: Long, + records: MemoryRecords): Unit = { + val endOffset = startOffset + records.deepEntries.asScala.size + val logMock = EasyMock.mock(classOf[Log]) + val fileRecordsMock = EasyMock.mock(classOf[FileRecords]) + + EasyMock.expect(replicaManager.getLog(groupMetadataTopicPartition)).andStubReturn(Some(logMock)) + EasyMock.expect(logMock.logStartOffset).andStubReturn(startOffset) + EasyMock.expect(replicaManager.getHighWatermark(groupMetadataTopicPartition)).andStubReturn(Some(endOffset)) + EasyMock.expect(logMock.read(EasyMock.eq(startOffset), EasyMock.anyInt(), EasyMock.eq(None), EasyMock.eq(true))) + .andReturn(FetchDataInfo(LogOffsetMetadata(startOffset), fileRecordsMock)) + EasyMock.expect(fileRecordsMock.readInto(EasyMock.anyObject(classOf[ByteBuffer]), EasyMock.anyInt())) + .andReturn(records.buffer) + + EasyMock.replay(logMock, fileRecordsMock) + } + + private def createCommittedOffsetRecords(committedOffsets: Map[TopicPartition, Long], + groupId: String = groupId): Seq[Record] = { + committedOffsets.map { case (topicPartition, offset) => + val offsetAndMetadata = OffsetAndMetadata(offset) + val offsetCommitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition) + val offsetCommitValue = GroupMetadataManager.offsetCommitValue(offsetAndMetadata) + Record.create(offsetCommitKey, offsetCommitValue) + }.toSeq } } diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala index bf695bf7eab1a..3db7818538118 100644 --- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataTest.scala @@ -180,14 +180,14 @@ class GroupMetadataTest extends JUnitSuite { val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) - group.add(memberId, member) + group.add(member) assertEquals("range", group.selectProtocol) val otherMemberId = "otherMemberId" val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) - group.add(otherMemberId, otherMember) + group.add(otherMember) // now could be either range or robin since there is no majority preference assertTrue(Set("range", "roundrobin")(group.selectProtocol)) @@ -195,7 +195,7 @@ class GroupMetadataTest extends JUnitSuite { val lastMember = new MemberMetadata(lastMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("range", Array.empty[Byte]))) - group.add(lastMemberId, lastMember) + group.add(lastMember) // now we should prefer 'roundrobin' assertEquals("roundrobin", group.selectProtocol) } @@ -216,8 +216,8 @@ class GroupMetadataTest extends JUnitSuite { val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) - group.add(memberId, member) - group.add(otherMemberId, otherMember) + group.add(member) + group.add(otherMember) assertEquals("roundrobin", group.selectProtocol) } @@ -230,7 +230,7 @@ class GroupMetadataTest extends JUnitSuite { val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("range", Array.empty[Byte]), ("roundrobin", Array.empty[Byte]))) - group.add(memberId, member) + group.add(member) assertTrue(group.supportsProtocols(Set("roundrobin", "foo"))) assertTrue(group.supportsProtocols(Set("range", "foo"))) assertFalse(group.supportsProtocols(Set("foo", "bar"))) @@ -239,7 +239,7 @@ class GroupMetadataTest extends JUnitSuite { val otherMember = new MemberMetadata(otherMemberId, groupId, clientId, clientHost, rebalanceTimeoutMs, sessionTimeoutMs, protocolType, List(("roundrobin", Array.empty[Byte]), ("blah", Array.empty[Byte]))) - group.add(otherMemberId, otherMember) + group.add(otherMember) assertTrue(group.supportsProtocols(Set("roundrobin", "foo"))) assertFalse(group.supportsProtocols(Set("range", "foo"))) @@ -253,7 +253,7 @@ class GroupMetadataTest extends JUnitSuite { group.transitionTo(PreparingRebalance) member.awaitingJoinCallback = _ => () - group.add(memberId, member) + group.add(member) assertEquals(0, group.generationId) assertNull(group.protocol) diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 270fca2e01624..3534d96152746 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -5,7 +5,7 @@ * 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 @@ -47,7 +47,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { * Implementations must override this method to return a set of KafkaConfigs. This method will be invoked for every * test and should not reuse previous configurations unless they select their ports randomly when servers are started. */ - def generateConfigs(): Seq[KafkaConfig] + def generateConfigs: Seq[KafkaConfig] /** * Override this in case ACLs or security credentials must be set before `servers` are started. @@ -64,7 +64,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { def configs: Seq[KafkaConfig] = { if (instanceConfigs == null) - instanceConfigs = generateConfigs() + instanceConfigs = generateConfigs instanceConfigs } @@ -75,7 +75,8 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT protected def listenerName: ListenerName = ListenerName.forSecurityProtocol(securityProtocol) protected def trustStoreFile: Option[File] = None - protected def saslProperties: Option[Properties] = None + protected def serverSaslProperties: Option[Properties] = None + protected def clientSaslProperties: Option[Properties] = None @Before override def setUp() { @@ -101,7 +102,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { } super.tearDown } - + /** * Pick a broker at random and kill it if it isn't already dead * Return the id of the broker killed @@ -119,7 +120,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { alive(index) = false } } - + /** * Restart any dead brokers */ diff --git a/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala new file mode 100644 index 0000000000000..19a0f9d776cd7 --- /dev/null +++ b/core/src/test/scala/unit/kafka/integration/MetricsDuringTopicCreationDeletionTest.scala @@ -0,0 +1,161 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.integration + +import java.util.Properties +import kafka.server.KafkaConfig +import kafka.utils.{Logging, TestUtils} +import scala.collection.JavaConverters.mapAsScalaMapConverter + +import org.junit.{Before, Test} +import com.yammer.metrics.Metrics +import com.yammer.metrics.core.Gauge + + +class MetricsDuringTopicCreationDeletionTest extends KafkaServerTestHarness with Logging { + + private val nodesNum = 3 + private val topicName = "topic" + private val topicNum = 2 + private val replicationFactor = 3 + private val partitionNum = 3 + private val createDeleteIterations = 3 + + private val overridingProps = new Properties + overridingProps.put(KafkaConfig.DeleteTopicEnableProp, "true") + overridingProps.put(KafkaConfig.AutoCreateTopicsEnableProp, "false") + // speed up the test for UnderReplicatedPartitions + // which relies on the ISR expiry thread to execute concurrently with topic creation + overridingProps.put(KafkaConfig.ReplicaLagTimeMaxMsProp, "2000") + + private val testedMetrics = List("OfflinePartitionsCount","PreferredReplicaImbalanceCount","UnderReplicatedPartitions") + private val topics = List.tabulate(topicNum) (n => topicName + n) + + @volatile private var running = true + + override def generateConfigs() = TestUtils.createBrokerConfigs(nodesNum, zkConnect) + .map(KafkaConfig.fromProps(_, overridingProps)) + + @Before + override def setUp { + // Do some Metrics Registry cleanup by removing the metrics that this test checks. + // This is a test workaround to the issue that prior harness runs may have left a populated registry. + // see https://issues.apache.org/jira/browse/KAFKA-4605 + for (m <- (testedMetrics)) { + Metrics.defaultRegistry.allMetrics.asScala + .filterKeys(k => k.getName.endsWith(m)) + .headOption match { + case Some(e) => Metrics.defaultRegistry.removeMetric(e._1) + case None => + } + } + + super.setUp + } + + /* + * checking all metrics we care in a single test is faster though it would be more elegant to have 3 @Test methods + */ + @Test + def testMetricsDuringTopicCreateDelete() { + + // For UnderReplicatedPartitions, because of https://issues.apache.org/jira/browse/KAFKA-4605 + // we can't access the metrics value of each server. So instead we directly invoke the method + // replicaManager.underReplicatedPartitionCount() that defines the metrics value. + @volatile var underReplicatedPartitionCount = 0 + + // For OfflinePartitionsCount and PreferredReplicaImbalanceCount even with https://issues.apache.org/jira/browse/KAFKA-4605 + // the test has worked reliably because the metric that gets triggered is the one generated by the first started server (controller) + val offlinePartitionsCountGauge = getGauge("OfflinePartitionsCount") + @volatile var offlinePartitionsCount = offlinePartitionsCountGauge.value + assert(offlinePartitionsCount == 0) + + val preferredReplicaImbalanceCountGauge = getGauge("PreferredReplicaImbalanceCount") + @volatile var preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value + assert(preferredReplicaImbalanceCount == 0) + + // Thread checking the metric continuously + running = true + val thread = new Thread(new Runnable { + def run() { + while (running) { + for ( s <- servers if running) { + underReplicatedPartitionCount = s.replicaManager.underReplicatedPartitionCount + if (underReplicatedPartitionCount > 0) { + running = false + } + } + + preferredReplicaImbalanceCount = preferredReplicaImbalanceCountGauge.value + if (preferredReplicaImbalanceCount > 0) { + running = false + } + + offlinePartitionsCount = offlinePartitionsCountGauge.value + if (offlinePartitionsCount > 0) { + running = false + } + } + } + }) + thread.start + + // breakable loop that creates and deletes topics + createDeleteTopics() + + // if the thread checking the gauge is still run, stop it + running = false; + thread.join + + assert(offlinePartitionsCount==0, "OfflinePartitionCount not 0: "+ offlinePartitionsCount) + assert(preferredReplicaImbalanceCount==0, "PreferredReplicaImbalanceCount not 0: " + preferredReplicaImbalanceCount) + assert(underReplicatedPartitionCount==0, "UnderReplicatedPartitionCount not 0: " + underReplicatedPartitionCount) + } + + private def getGauge(metricName: String) = { + Metrics.defaultRegistry.allMetrics.asScala + .filterKeys(k => k.getName.endsWith(metricName)) + .headOption + .getOrElse { fail( "Unable to find metric " + metricName ) } + ._2.asInstanceOf[Gauge[Int]] + } + + private def createDeleteTopics() { + for (l <- 1 to createDeleteIterations if running) { + // Create topics + for (t <- topics if running) { + try { + kafka.admin.AdminUtils.createTopic(zkUtils, t, partitionNum, replicationFactor) + } catch { + case e: Exception => e.printStackTrace + } + } + Thread.sleep(500) + + // Delete topics + for (t <- topics if running) { + try { + kafka.admin.AdminUtils.deleteTopic(zkUtils, t) + } catch { + case e: Exception => e.printStackTrace + } + } + Thread.sleep(500) + } + } +} diff --git a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala index 3327a650428c3..a53602db83325 100644 --- a/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala +++ b/core/src/test/scala/unit/kafka/javaapi/message/BaseMessageSetTestCases.scala @@ -40,14 +40,14 @@ trait BaseMessageSetTestCases extends JUnitSuite { def testIteratorIsConsistent() { val m = createMessageSet(messages) // two iterators over the same set should give the same results - TestUtils.checkEquals(m.iterator, m.iterator) + TestUtils.checkEquals(m, m) } @Test def testIteratorIsConsistentWithCompression() { val m = createMessageSet(messages, DefaultCompressionCodec) // two iterators over the same set should give the same results - TestUtils.checkEquals(m.iterator, m.iterator) + TestUtils.checkEquals(m, m) } @Test diff --git a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala index 862083daff50a..66702d683abcb 100644 --- a/core/src/test/scala/unit/kafka/log/LogConfigTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogConfigTest.scala @@ -111,16 +111,6 @@ class LogConfigTest { case _: ConfigException => false } } - def testValueValidator() { - val p = new Properties() - p.setProperty(LogConfig.SegmentBytesProp, "100") - p.setProperty(LogConfig.RetentionBytesProp, "100") - LogConfig.validate(p) - p.setProperty(LogConfig.RetentionBytesProp, "90") - val except = intercept[IllegalArgumentException] { - LogConfig.validate(p) - } - } private def assertPropertyInvalid(name: String, values: AnyRef*) { values.foreach((value) => { diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 08cdac5807176..9e0deb2dc075a 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -659,8 +659,10 @@ class LogTest extends JUnitSuite { // The rebuilt time index should be empty log = new Log(logDir, config, recoveryPoint = numMessages + 1, time.scheduler, time) val segArray = log.logSegments.toArray - for (i <- 0 until segArray.size - 1) + for (i <- 0 until segArray.size - 1) { assertEquals("The time index should be empty", 0, segArray(i).timeIndex.entries) + assertEquals("The time index file size should be 0", 0, segArray(i).timeIndex.file.length) + } } diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index 7a00f2a7b6998..6358bdc83a00a 100755 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -21,7 +21,7 @@ import java.util.Properties import java.util.concurrent.LinkedBlockingQueue import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} -import org.junit.Assert._ +import org.junit.Assert.{assertEquals, assertTrue} import org.easymock.EasyMock import org.junit.Test import kafka.api._ @@ -36,7 +36,6 @@ import kafka.utils.TestUtils._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import kafka.utils._ -import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.utils.Time @deprecated("This test has been deprecated and it will be removed in a future release.", "0.10.0.0") diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 8cec0c79c8a4f..ba1becc63fa93 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -83,7 +83,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { val aclList = zkUtils.zkConnection.getAcl(path).getKey assertTrue(aclList.size == 2) for (acl: ACL <- aclList.asScala) { - assertTrue(isAclSecure(acl)) + assertTrue(isAclSecure(acl, false)) } } } @@ -153,12 +153,12 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { @Test def testDeleteRecursive() { info(s"zkConnect string: $zkConnect") - for (path <- zkUtils.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths) { info(s"Creating $path") zkUtils.makeSurePersistentPathExists(path) zkUtils.createPersistentPath(s"$path/fpjwashere", "") } - zkUtils.zkConnection.setAcl("/", zkUtils.DefaultAcls, -1) + zkUtils.zkConnection.setAcl("/", zkUtils.defaultAcls("/"), -1) deleteAllUnsecure() } @@ -185,7 +185,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { */ private def testMigration(zkUrl: String, firstZk: ZkUtils, secondZk: ZkUtils) { info(s"zkConnect string: $zkUrl") - for (path <- firstZk.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths ++ ZkUtils.SensitiveZkRootPaths) { info(s"Creating $path") firstZk.makeSurePersistentPathExists(path) // Create a child for each znode to exercise the recurrent @@ -206,39 +206,41 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { } ZkSecurityMigrator.run(Array(s"--zookeeper.acl=$secureOpt", s"--zookeeper.connect=$zkUrl")) info("Done with migration") - for (path <- secondZk.securePersistentZkPaths) { + for (path <- ZkUtils.SecureZkRootPaths ++ ZkUtils.SensitiveZkRootPaths) { + val sensitive = ZkUtils.sensitivePath(path) val listParent = secondZk.zkConnection.getAcl(path).getKey - assertTrue(path, isAclCorrect(listParent, secondZk.isSecure)) + assertTrue(path, isAclCorrect(listParent, secondZk.isSecure, sensitive)) val childPath = path + "/fpjwashere" val listChild = secondZk.zkConnection.getAcl(childPath).getKey - assertTrue(childPath, isAclCorrect(listChild, secondZk.isSecure)) + assertTrue(childPath, isAclCorrect(listChild, secondZk.isSecure, sensitive)) } // Check consumers path. val consumersAcl = firstZk.zkConnection.getAcl(ZkUtils.ConsumersPath).getKey - assertTrue(ZkUtils.ConsumersPath, isAclCorrect(consumersAcl, false)) + assertTrue(ZkUtils.ConsumersPath, isAclCorrect(consumersAcl, false, false)) } /** * Verifies that the path has the appropriate secure ACL. */ private def verify(path: String): Boolean = { + val sensitive = ZkUtils.sensitivePath(path) val list = zkUtils.zkConnection.getAcl(path).getKey - list.asScala.forall(isAclSecure) + list.asScala.forall(isAclSecure(_, sensitive)) } /** * Verifies ACL. */ - private def isAclCorrect(list: java.util.List[ACL], secure: Boolean): Boolean = { + private def isAclCorrect(list: java.util.List[ACL], secure: Boolean, sensitive: Boolean): Boolean = { val isListSizeCorrect = - if (secure) + if (secure && !sensitive) list.size == 2 else list.size == 1 isListSizeCorrect && list.asScala.forall( if (secure) - isAclSecure + isAclSecure(_, sensitive) else isAclUnsecure ) @@ -249,10 +251,10 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { * values are based on the constants used in the * ZooKeeper code base. */ - private def isAclSecure(acl: ACL): Boolean = { + private def isAclSecure(acl: ACL, sensitive: Boolean): Boolean = { info(s"ACL $acl") acl.getPerms match { - case 1 => acl.getId.getScheme.equals("world") + case 1 => !sensitive && acl.getId.getScheme.equals("world") case 31 => acl.getId.getScheme.equals("sasl") case _ => false } diff --git a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala index 76774bdf55be3..32e69c772b39d 100644 --- a/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/AbstractCreateTopicsRequestTest.scala @@ -130,7 +130,7 @@ class AbstractCreateTopicsRequestTest extends BaseRequestTest { } protected def replicaAssignmentToJava(assignments: Map[Int, List[Int]]) = { - assignments.map { case (k, v) => (k:Integer, v.map { i => i:Integer }.asJava) }.asJava + assignments.map { case (k, v) => (k: Integer, v.map { i => i: Integer }.asJava) }.asJava } protected def sendCreateTopicRequest(request: CreateTopicsRequest, socketServer: SocketServer = controllerSocketServer): CreateTopicsResponse = { diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 3825489ddb737..b864e5d0875fb 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -42,7 +42,7 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false, enableDeleteTopic = true, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile, saslProperties = saslProperties) + trustStoreFile = trustStoreFile, saslProperties = serverSaslProperties) props.foreach(propertyOverrides) props.map(KafkaConfig.fromProps) } diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala index e3b0bbe504939..6efa189a6c828 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestTest.scala @@ -63,7 +63,7 @@ class CreateTopicsRequestTest extends AbstractCreateTopicsRequestTest { // Basic validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder(Map(existingTopic -> new CreateTopicsRequest.TopicDetails(1, 1.toShort)).asJava, timeout).build(), - Map(existingTopic -> error(Errors.TOPIC_ALREADY_EXISTS, Some("""Topic "existing-topic" already exists.""")))) + Map(existingTopic -> error(Errors.TOPIC_ALREADY_EXISTS, Some("Topic 'existing-topic' already exists.")))) validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder(Map("error-partitions" -> new CreateTopicsRequest.TopicDetails(-1, 1.toShort)).asJava, timeout).build(), Map("error-partitions" -> error(Errors.INVALID_PARTITIONS)), checkErrorMessage = false) validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder(Map("error-replication" -> new CreateTopicsRequest.TopicDetails(1, (numBrokers + 1).toShort)).asJava, timeout).build(), diff --git a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala index 7127eaffca9de..f2020cf8bb41e 100644 --- a/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala +++ b/core/src/test/scala/unit/kafka/server/CreateTopicsRequestWithPolicyTest.scala @@ -17,8 +17,10 @@ package kafka.server +import java.util import java.util.Properties +import kafka.log.LogConfig import kafka.utils.TestUtils import org.apache.kafka.common.errors.PolicyViolationException import org.apache.kafka.common.protocol.Errors @@ -34,16 +36,26 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest override def propertyOverrides(properties: Properties): Unit = { super.propertyOverrides(properties) - properties.put(KafkaConfig.CreateTopicsPolicyClassNameProp, classOf[Policy].getName) + properties.put(KafkaConfig.CreateTopicPolicyClassNameProp, classOf[Policy].getName) } @Test def testValidCreateTopicsRequests() { val timeout = 10000 + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("topic1" -> new CreateTopicsRequest.TopicDetails(5, 1.toShort)).asJava, timeout).build()) + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("topic2" -> new CreateTopicsRequest.TopicDetails(5, 3.toShort)).asJava, timeout, true).build()) + + val configs = Map(LogConfig.RetentionMsProp -> 4999.toString) + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("topic3" -> new CreateTopicsRequest.TopicDetails(11, 2.toShort, configs.asJava)).asJava, timeout, true).build()) + + val assignments = replicaAssignmentToJava(Map(0 -> List(1, 0), 1 -> List(0, 1))) + validateValidCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("topic4" -> new CreateTopicsRequest.TopicDetails(assignments)).asJava, timeout).build()) } @Test @@ -54,28 +66,90 @@ class CreateTopicsRequestWithPolicyTest extends AbstractCreateTopicsRequestTest // Policy violations validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( - Map("topic3" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout).build(), - Map("topic3" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + Map("policy-topic1" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout).build(), + Map("policy-topic1" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("policy-topic2" -> new CreateTopicsRequest.TopicDetails(4, 3.toShort)).asJava, timeout, true).build(), + Map("policy-topic2" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + + val configs = Map(LogConfig.RetentionMsProp -> 5001.toString) + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("policy-topic3" -> new CreateTopicsRequest.TopicDetails(11, 2.toShort, configs.asJava)).asJava, timeout, true).build(), + Map("policy-topic3" -> error(Errors.POLICY_VIOLATION, Some("RetentionMs should be less than 5000ms if replicationFactor > 5")))) validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( - Map("topic4" -> new CreateTopicsRequest.TopicDetails(4, 1.toShort)).asJava, timeout, true).build(), - Map("topic4" -> error(Errors.POLICY_VIOLATION, Some("Topics should have at least 5 partitions, received 4")))) + Map("policy-topic4" -> new CreateTopicsRequest.TopicDetails(11, 3.toShort, Map.empty.asJava)).asJava, timeout, true).build(), + Map("policy-topic4" -> error(Errors.POLICY_VIOLATION, Some("RetentionMs should be less than 5000ms if replicationFactor > 5")))) + + val assignments = replicaAssignmentToJava(Map(0 -> List(1), 1 -> List(0))) + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("policy-topic5" -> new CreateTopicsRequest.TopicDetails(assignments)).asJava, timeout).build(), + Map("policy-topic5" -> error(Errors.POLICY_VIOLATION, + Some("Topic partitions should have at least 2 partitions, received 1 for partition 0")))) // Check that basic errors still work validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( Map(existingTopic -> new CreateTopicsRequest.TopicDetails(5, 1.toShort)).asJava, timeout).build(), - Map(existingTopic -> error(Errors.TOPIC_ALREADY_EXISTS, Some("""Topic "existing-topic" already exists.""")))) + Map(existingTopic -> error(Errors.TOPIC_ALREADY_EXISTS, Some("Topic 'existing-topic' already exists.")))) + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( Map("error-replication" -> new CreateTopicsRequest.TopicDetails(10, (numBrokers + 1).toShort)).asJava, timeout, true).build(), - Map("error-replication" -> error(Errors.INVALID_REPLICATION_FACTOR, Some("replication factor: 4 larger than available brokers: 3")))) + Map("error-replication" -> error(Errors.INVALID_REPLICATION_FACTOR, + Some("replication factor: 4 larger than available brokers: 3")))) + + validateErrorCreateTopicsRequests(new CreateTopicsRequest.Builder( + Map("error-replication2" -> new CreateTopicsRequest.TopicDetails(10, -1: Short)).asJava, timeout, true).build(), + Map("error-replication2" -> error(Errors.INVALID_REPLICATION_FACTOR, Some("replication factor must be larger than 0")))) } } object CreateTopicsRequestWithPolicyTest { + class Policy extends CreateTopicPolicy { - def validate(requestMetadata: RequestMetadata): Unit = - if (requestMetadata.numPartitions < 5) - throw new PolicyViolationException(s"Topics should have at least 5 partitions, received ${requestMetadata.numPartitions}") + + var configs: Map[String, _] = _ + var closed = false + + def configure(configs: util.Map[String, _]): Unit = { + this.configs = configs.asScala.toMap + } + + def validate(requestMetadata: RequestMetadata): Unit = { + require(!closed, "Policy should not be closed") + require(!configs.isEmpty, "configure should have been called with non empty configs") + + import requestMetadata._ + if (numPartitions != null || replicationFactor != null) { + require(numPartitions != null, s"numPartitions should not be null, but it is $numPartitions") + require(replicationFactor != null, s"replicationFactor should not be null, but it is $replicationFactor") + require(replicasAssignments == null, s"replicaAssigments should be null, but it is $replicasAssignments") + + if (numPartitions < 5) + throw new PolicyViolationException(s"Topics should have at least 5 partitions, received $numPartitions") + + if (numPartitions > 10) { + if (requestMetadata.configs.asScala.get(LogConfig.RetentionMsProp).fold(true)(_.toInt > 5000)) + throw new PolicyViolationException("RetentionMs should be less than 5000ms if replicationFactor > 5") + } else + require(requestMetadata.configs.isEmpty, s"Topic configs should be empty, but it is ${requestMetadata.configs}") + + } else { + require(numPartitions == null, s"numPartitions should be null, but it is $numPartitions") + require(replicationFactor == null, s"replicationFactor should be null, but it is $replicationFactor") + require(replicasAssignments != null, s"replicaAssigments should not be null, but it is $replicasAssignments") + + replicasAssignments.asScala.foreach { case (partitionId, assignment) => + if (assignment.size < 2) + throw new PolicyViolationException("Topic partitions should have at least 2 partitions, received " + + s"${assignment.size} for partition $partitionId") + } + } + + } + + def close(): Unit = closed = true + } } diff --git a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala index cf0dc6fd6cdc1..dc30fb20eb0cd 100644 --- a/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala +++ b/core/src/test/scala/unit/kafka/server/DynamicConfigChangeTest.scala @@ -148,7 +148,7 @@ class DynamicConfigChangeTest extends KafkaServerTestHarness { AdminUtils.changeUserOrUserClientIdConfig(zkUtils, "ANONYMOUS/clients/overriddenUserClientId", userClientIdProps) // Remove config change znodes to force quota initialization only through loading of user/client quotas - zkUtils.getChildren(ZkUtils.EntityConfigChangesPath).foreach { p => zkUtils.deletePath(ZkUtils.EntityConfigChangesPath + "/" + p) } + zkUtils.getChildren(ZkUtils.ConfigChangesPath).foreach { p => zkUtils.deletePath(ZkUtils.ConfigChangesPath + "/" + p) } server.startup() val quotaManagers = server.apis.quotas diff --git a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala index 5c53ffa167103..1f9e18ba805fd 100755 --- a/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/EdgeCaseRequestTest.scala @@ -27,7 +27,7 @@ import kafka.utils._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.types.Type -import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.record.MemoryRecords import org.apache.kafka.common.requests.{ProduceRequest, ProduceResponse, ResponseHeader} import org.junit.Assert._ @@ -138,7 +138,7 @@ class EdgeCaseRequestTest extends KafkaServerTestHarness { val partitionResponse = produceResponse.responses().get(topicPartition) assertNotNull(partitionResponse) - assertEquals("There should be no error", 0, partitionResponse.errorCode) + assertEquals("There should be no error", Errors.NONE, partitionResponse.error) } @Test diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index a5ca5a832e226..0f5ff5dec6300 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -24,6 +24,7 @@ import kafka.cluster.EndPoint import kafka.message._ import kafka.utils.{CoreUtils, TestUtils} import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.metrics.Sensor import org.apache.kafka.common.network.ListenerName import org.apache.kafka.common.protocol.SecurityProtocol import org.junit.Assert._ @@ -547,7 +548,7 @@ class KafkaConfigTest { case KafkaConfig.RequestTimeoutMsProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.AuthorizerClassNameProp => //ignore string - case KafkaConfig.CreateTopicsPolicyClassNameProp => //ignore string + case KafkaConfig.CreateTopicPolicyClassNameProp => //ignore string case KafkaConfig.PortProp => assertPropertyInvalid(getBaseProperties(), name, "not_a_number") case KafkaConfig.HostNameProp => // ignore string @@ -684,6 +685,8 @@ class KafkaConfigTest { //For LogFlushIntervalMsProp defaults.put(KafkaConfig.LogFlushSchedulerIntervalMsProp, "123") defaults.put(KafkaConfig.OffsetsTopicCompressionCodecProp, SnappyCompressionCodec.codec.toString) + // For MetricRecordingLevelProp + defaults.put(KafkaConfig.MetricRecordingLevelProp, Sensor.RecordingLevel.DEBUG.toString) val config = KafkaConfig.fromProps(defaults) assertEquals("127.0.0.1:2181", config.zkConnect) @@ -701,6 +704,7 @@ class KafkaConfigTest { assertEquals(10 * 60L * 1000L * 60, config.logRetentionTimeMillis) assertEquals(123L, config.logFlushIntervalMs) assertEquals(SnappyCompressionCodec, config.offsetsTopicCompressionCodec) + assertEquals(Sensor.RecordingLevel.DEBUG.toString, config.metricRecordingLevel) } private def assertPropertyInvalid(validRequiredProps: => Properties, name: String, values: Any*) { diff --git a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala index d235d02a10210..dfcb4ac279990 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaMetricReporterClusterIdTest.scala @@ -28,6 +28,7 @@ import org.junit.{After, Before, Test} import org.apache.kafka.test.TestUtils.isValidClusterId object KafkaMetricReporterClusterIdTest { + val setupError = new AtomicReference[String]("") class MockKafkaMetricsReporter extends KafkaMetricsReporter with ClusterResourceListener { @@ -52,8 +53,26 @@ object KafkaMetricReporterClusterIdTest { override def onUpdate(clusterMetadata: ClusterResource) { MockBrokerMetricsReporter.CLUSTER_META.set(clusterMetadata) } - } + override def configure(configs: java.util.Map[String, _]): Unit = { + // Check that the configuration passed to the MetricsReporter includes the broker id as an Integer. + // This is a regression test for KAFKA-4756. + // + // Because this code is run during the test setUp phase, if we throw an exception here, + // it just results in the test itself being declared "not found" rather than failing. + // So we track an error message which we will check later in the test body. + val brokerId = configs.get(KafkaConfig.BrokerIdProp) + if (brokerId == null) + setupError.compareAndSet("", "No value was set for the broker id.") + else if (!brokerId.isInstanceOf[String]) + setupError.compareAndSet("", "The value set for the broker id was not a string.") + try + Integer.parseInt(brokerId.asInstanceOf[String]) + catch { + case e: Exception => setupError.compareAndSet("", "Error parsing broker id " + e.toString) + } + } + } } class KafkaMetricReporterClusterIdTest extends ZooKeeperTestHarness { @@ -66,6 +85,8 @@ class KafkaMetricReporterClusterIdTest extends ZooKeeperTestHarness { val props = TestUtils.createBrokerConfig(1, zkConnect) props.setProperty("kafka.metrics.reporters", "kafka.server.KafkaMetricReporterClusterIdTest$MockKafkaMetricsReporter") props.setProperty(KafkaConfig.MetricReporterClassesProp, "kafka.server.KafkaMetricReporterClusterIdTest$MockBrokerMetricsReporter") + props.setProperty(KafkaConfig.BrokerIdGenerationEnableProp, "true") + props.setProperty(KafkaConfig.BrokerIdProp, "-1") config = KafkaConfig.fromProps(props) server = KafkaServerStartable.fromProps(props) server.startup() @@ -73,6 +94,8 @@ class KafkaMetricReporterClusterIdTest extends ZooKeeperTestHarness { @Test def testClusterIdPresent() { + assertEquals("", KafkaMetricReporterClusterIdTest.setupError.get()) + assertNotNull(KafkaMetricReporterClusterIdTest.MockKafkaMetricsReporter.CLUSTER_META) isValidClusterId(KafkaMetricReporterClusterIdTest.MockKafkaMetricsReporter.CLUSTER_META.get().clusterId()) diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 6ca784a272486..355acde17255a 100755 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -30,6 +30,8 @@ import org.junit.Assert._ import java.util.Properties import java.io.File +import kafka.admin.AdminUtils + import scala.util.Random import scala.collection._ @@ -46,7 +48,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness { @Before override def setUp() { super.setUp() - val config: Properties = createBrokerConfig(1, zkConnect) + val config: Properties = createBrokerConfig(1, zkConnect, enableDeleteTopic = true) config.setProperty(KafkaConfig.OffsetsTopicReplicationFactorProp, "1") config.setProperty(KafkaConfig.OffsetsRetentionCheckIntervalMsProp, retentionCheckInterval.toString) val logDirPath = config.getProperty("log.dir") @@ -304,4 +306,29 @@ class OffsetCommitTest extends ZooKeeperTestHarness { assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, commitResponse.commitStatus.get(TopicAndPartition(topic1, 0)).get) assertEquals(Errors.NONE.code, commitResponse.commitStatus.get(TopicAndPartition(topic2, 0)).get) } + + @Test + def testOffsetsDeleteAfterTopicDeletion() { + // set up topic partition + val topic = "topic" + val topicPartition = TopicAndPartition(topic, 0) + createTopic(zkUtils, topic, servers = Seq(server), numPartitions = 1) + + val commitRequest = OffsetCommitRequest(group, immutable.Map(topicPartition -> OffsetAndMetadata(offset = 42L))) + val commitResponse = simpleConsumer.commitOffsets(commitRequest) + + assertEquals(Errors.NONE.code, commitResponse.commitStatus.get(topicPartition).get) + + // start topic deletion + AdminUtils.deleteTopic(zkUtils, topic) + TestUtils.verifyTopicDeletion(zkUtils, topic, 1, Seq(server)) + Thread.sleep(retentionCheckInterval * 2) + + // check if offsets deleted + val fetchRequest = OffsetFetchRequest(group, Seq(TopicAndPartition(topic, 0))) + val offsetMetadataAndErrorMap = simpleConsumer.fetchOffsets(fetchRequest) + val offsetMetadataAndError = offsetMetadataAndErrorMap.requestInfo(topicPartition) + assertEquals(OffsetMetadataAndError.NoOffset, offsetMetadataAndError) + } + } diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala index 8ed93d94d3f4b..b05be9d580531 100644 --- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala @@ -45,9 +45,9 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals(1, produceResponse.responses.size) val (tp, partitionResponse) = produceResponse.responses.asScala.head assertEquals(topicPartition, tp) - assertEquals(Errors.NONE.code, partitionResponse.errorCode) + assertEquals(Errors.NONE, partitionResponse.error) assertEquals(expectedOffset, partitionResponse.baseOffset) - assertEquals(-1, partitionResponse.timestamp) + assertEquals(-1, partitionResponse.logAppendTime) partitionResponse } @@ -82,9 +82,9 @@ class ProduceRequestTest extends BaseRequestTest { assertEquals(1, produceResponse.responses.size) val (tp, partitionResponse) = produceResponse.responses.asScala.head assertEquals(topicPartition, tp) - assertEquals(Errors.CORRUPT_MESSAGE.code, partitionResponse.errorCode) + assertEquals(Errors.CORRUPT_MESSAGE, partitionResponse.error) assertEquals(-1, partitionResponse.baseOffset) - assertEquals(-1, partitionResponse.timestamp) + assertEquals(-1, partitionResponse.logAppendTime) } private def sendProduceRequest(leaderId: Int, request: ProduceRequest): ProduceResponse = { diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 00959f142a413..d5075aaa7a0dc 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -101,7 +101,7 @@ class ReplicaManagerTest { new AtomicBoolean(false), QuotaFactory.instantiate(config, metrics, time).follower, Option(this.getClass.getName)) try { def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = { - assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code) + assert(responseStatus.values.head.error == Errors.INVALID_REQUIRED_ACKS) } rm.appendRecords( timeout = 0, @@ -128,13 +128,15 @@ class ReplicaManagerTest { try { var produceCallbackFired = false def produceCallback(responseStatus: Map[TopicPartition, PartitionResponse]) = { - assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.errorCode) + assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION, + responseStatus.values.head.error) produceCallbackFired = true } var fetchCallbackFired = false def fetchCallback(responseStatus: Seq[(TopicPartition, FetchPartitionData)]) = { - assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error) + assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION, + responseStatus.map(_._2).head.error) fetchCallbackFired = true } @@ -229,7 +231,7 @@ class ReplicaManagerTest { var fetchError = 0 var fetchedRecords: Records = null def fetchCallback(responseStatus: Seq[(TopicPartition, FetchPartitionData)]) = { - fetchError = responseStatus.map(_._2).head.error + fetchError = responseStatus.map(_._2).head.error.code fetchedRecords = responseStatus.map(_._2).head.records fetchCallbackFired = true } diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala index 34d7d147c1cb7..07e03e3824ef9 100644 --- a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -33,7 +33,8 @@ class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT override protected val kafkaClientSaslMechanism = "PLAIN" override protected val kafkaServerSaslMechanisms = List("PLAIN") - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val serverSaslProperties = Some(kafkaServerSaslProperties(kafkaServerSaslMechanisms, kafkaClientSaslMechanism)) + override protected val clientSaslProperties = Some(kafkaClientSaslProperties(kafkaClientSaslMechanism)) override protected val zkSaslEnabled = false override def numBrokers = 1 diff --git a/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala b/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala new file mode 100755 index 0000000000000..dc96680c95893 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ServerMetricsTest.scala @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package kafka.server + +import kafka.utils.TestUtils +import org.apache.kafka.common.metrics.Sensor +import org.junit.Assert._ +import org.junit.Test +import org.scalatest.junit.JUnitSuite + +class ServerMetricsTest extends JUnitSuite { + + @Test + def testMetricsConfig(): Unit = { + val recordingLevels = List(Sensor.RecordingLevel.DEBUG, Sensor.RecordingLevel.INFO) + val illegalNames = List("IllegalName", "") + val props = TestUtils.createBrokerConfig(0, "localhost:2818") + + for (recordingLevel <- recordingLevels) { + props.put(KafkaConfig.MetricRecordingLevelProp, recordingLevel.name) + val config = KafkaConfig.fromProps(props) + val metricConfig = KafkaServer.metricConfig(config) + assertEquals(recordingLevel, metricConfig.recordLevel) + } + + for (illegalName <- illegalNames) { + intercept[IllegalArgumentException] { + props.put(KafkaConfig.MetricRecordingLevelProp, illegalName) + val config = KafkaConfig.fromProps(props) + KafkaServer.metricConfig(config) + } + } + + } + +} diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala index 7055b7e2b835b..0949eb793b4e5 100644 --- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala @@ -128,16 +128,16 @@ object JaasTestUtils { jaasFile.getCanonicalPath } - def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { + def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanism, clientKeyTabLocation)) writeToFile(jaasFile, kafkaSections) jaasFile.getCanonicalPath } - def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { + def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanism: Option[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanism, clientKeyTabLocation)) writeToFile(jaasFile, kafkaSections ++ zkSections) jaasFile.getCanonicalPath } @@ -209,9 +209,9 @@ object JaasTestUtils { /* * Used for the static JAAS configuration and it uses the credentials for client#2 */ - private def kafkaClientSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = { - new JaasSection(KafkaClientContextName, mechanisms.map(m => - kafkaClientModule(m, keytabLocation, KafkaClientPrincipal2, KafkaPlainUser2, KafkaPlainPassword2, KafkaScramUser2, KafkaScramPassword2))) + private def kafkaClientSection(mechanism: Option[String], keytabLocation: Option[File]): JaasSection = { + new JaasSection(KafkaClientContextName, mechanism.map(m => + kafkaClientModule(m, keytabLocation, KafkaClientPrincipal2, KafkaPlainUser2, KafkaPlainPassword2, KafkaScramUser2, KafkaScramPassword2)).toSeq) } private def jaasSectionsToString(jaasSections: Seq[JaasSection]): String = diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index c530e07604c2d..d4173ef9fcff3 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -288,6 +288,11 @@ object TestUtils extends Logging { props } + /** + * Fail a test case explicitly. Return Nothing so that we are not constrained by the return type. + */ + def fail(msg: String): Nothing = throw new AssertionError(msg) + /** * Wrap a single record log buffer. */ diff --git a/docs/api.html b/docs/api.html index 20c3642d00e9e..de0bb1d13a09b 100644 --- a/docs/api.html +++ b/docs/api.html @@ -31,7 +31,7 @@

    2.1 Producer API

    The Producer API allows applications to send streams of data to topics in the Kafka cluster.

    Examples showing how to use the producer are given in the - javadocs. + javadocs.

    To use the producer, you can use the following maven dependency: @@ -39,7 +39,7 @@

    2.1 Producer API

    <dependency> <groupId>org.apache.kafka</groupId> <artifactId>kafka-clients</artifactId> - <version>0.10.0.0</version> + <version>0.10.2.0</version> </dependency> @@ -48,14 +48,14 @@

    2.2 Consumer API

    The Consumer API allows applications to read streams of data from topics in the Kafka cluster.

    Examples showing how to use the consumer are given in the - javadocs. + javadocs.

    To use the consumer, you can use the following maven dependency:

     		<dependency>
     			<groupId>org.apache.kafka</groupId>
     			<artifactId>kafka-clients</artifactId>
    -			<version>0.10.0.0</version>
    +			<version>0.10.2.0</version>
     		</dependency>
     	
    @@ -64,9 +64,9 @@

    2.3 Streams API

    The Streams API allows transforming streams of data from input topics to output topics.

    Examples showing how to use this library are given in the - javadocs + javadocs

    - Additional documentation on using the Streams API is available here. + Additional documentation on using the Streams API is available here.

    To use Kafka Streams you can use the following maven dependency: @@ -74,7 +74,7 @@

    2.3 Streams API

    <dependency> <groupId>org.apache.kafka</groupId> <artifactId>kafka-streams</artifactId> - <version>0.10.0.0</version> + <version>0.10.2.0</version> </dependency> @@ -84,7 +84,7 @@

    2.4 Connect API

    Many users of Connect won't need to use this API directly, though, they can use pre-built connectors without needing to write any code. Additional information on using Connect is available here.

    - Those who want to implement custom connectors can see the javadoc. + Those who want to implement custom connectors can see the javadoc.

    2.5 Legacy APIs

    diff --git a/docs/connect.html b/docs/connect.html index 23e168cae0309..6ad06b8beb1b0 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -100,6 +100,88 @@

    Configuring Connecto For any other options, you should consult the documentation for the connector. +

    Transformations

    + + Connectors can be configured with transformations to make lightweight message-at-a-time modifications. They can be convenient for data massaging and event routing. + + A transformation chain can be specified in the connector configuration. + +
      +
    • transforms - List of aliases for the transformation, specifying the order in which the transformations will be applied.
    • +
    • transforms.$alias.type - Fully qualified class name for the transformation.
    • +
    • transforms.$alias.$transformationSpecificConfig Configuration properties for the transformation
    • +
    + +

    For example, lets take the built-in file source connector and use a transformation to add a static field.

    + +

    Throughout the example we'll use schemaless JSON data format. To use schemaless format, we changed the following two lines in connect-standalone.properties from true to false:

    + +
    +        key.converter.schemas.enable
    +        value.converter.schemas.enable
    +    
    + + The file source connector reads each line as a String. We will wrap each line in a Map and then add a second field to identify the origin of the event. To do this, we use two transformations: +
      +
    • HoistField to place the input line inside a Map
    • +
    • InsertField to add the static field. In this example we'll indicate that the record came from a file connector
    • +
    + + After adding the transformations, connect-file-source.properties file looks as following: + +
    +        name=local-file-source
    +        connector.class=FileStreamSource
    +        tasks.max=1
    +        file=test.txt
    +        topic=connect-test
    +        transforms=MakeMap, InsertSource
    +        transforms.MakeMap.type=org.apache.kafka.connect.transforms.HoistField$Value
    +        transforms.MakeMap.field=line
    +        transforms.InsertSource.type=org.apache.kafka.connect.transforms.InsertField$Value
    +        transforms.InsertSource.static.field=data_source
    +        transforms.InsertSource.static.value=test-file-source
    +    
    + +

    All the lines starting with transforms were added for the transformations. You can see the two transformations we created: "InsertSource" and "MakeMap" are aliases that we chose to give the transformations. The transformation types are based on the list of built-in transformations you can see below. Each transformation type has additional configuration: HoistField requires a configuration called "field", which is the name of the field in the map that will include the original String from the file. InsertField transformation lets us specify the field name and the value that we are adding.

    + + When we ran the file source connector on my sample file without the transformations, and then read them using kafka-console-consumer.sh, the results were: + +
    +        "foo"
    +        "bar"
    +        "hello world"
    +   
    + + We then create a new file connector, this time after adding the transformations to the configuration file. This time, the results will be: + +
    +        {"line":"foo","data_source":"test-file-source"}
    +        {"line":"bar","data_source":"test-file-source"}
    +        {"line":"hello world","data_source":"test-file-source"}
    +    
    + + You can see that the lines we've read are now part of a JSON map, and there is an extra field with the static value we specified. This is just one example of what you can do with transformations. + + Several widely-applicable data and routing transformations are included with Kafka Connect: + +
      +
    • InsertField - Add a field using either static data or record metadata
    • +
    • ReplaceField - Filter or rename fields
    • +
    • MaskField - Replace field with valid null value for the type (0, empty string, etc)
    • +
    • ValueToKey
    • +
    • HoistField - Wrap the entire event as a single field inside a Struct or a Map
    • +
    • ExtractField - Extract a specific field from Struct and Map and include only this field in results
    • +
    • SetSchemaMetadata - modify the schema name or version
    • +
    • TimestampRouter - Modify the topic of a record based on original topic and timestamp. Useful when using a sink that needs to write to different tables or indexes based on timestamps
    • +
    • RegexpRouter - modify the topic of a record based on original topic, replacement string and a regular expression
    • +
    + + Details on how to configure each transformation are listed below: + + + +

    REST API

    Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. By default, this service runs on port 8083. The following are the currently supported endpoints: diff --git a/docs/documentation.html b/docs/documentation.html index e8516f909e40c..f9ab6738c663a 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -20,141 +20,16 @@ +

    Documentation

    -

    Kafka 0.10.0 Documentation

    - Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X. - - - +

    Kafka 0.10.2 Documentation

    + Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X, 0.10.0.X, 0.10.1.X. + +

    1. Getting Started

    1.1 Introduction

    @@ -194,8 +69,15 @@

    7. Security

    8. Kafka Connect

    -

    9. Kafka Streams

    - +

    9. Kafka Streams

    +

    + Kafka Streams is a client library for processing and analyzing data stored in Kafka and either write the resulting data back to Kafka or send the final output to an external system. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management of application state. +

    +

    + Kafka Streams has a low barrier to entry: You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model. +

    + +

    Learn More about Kafka Streams read this Section.

    - + diff --git a/docs/documentation/streams.html b/docs/documentation/streams.html new file mode 100644 index 0000000000000..d8d2bb21779dc --- /dev/null +++ b/docs/documentation/streams.html @@ -0,0 +1,19 @@ + + + + diff --git a/docs/images/streams-architecture-overview.jpg b/docs/images/streams-architecture-overview.jpg new file mode 100644 index 0000000000000..92220791aa70a Binary files /dev/null and b/docs/images/streams-architecture-overview.jpg differ diff --git a/docs/images/streams-architecture-states.jpg b/docs/images/streams-architecture-states.jpg new file mode 100644 index 0000000000000..fde12db522475 Binary files /dev/null and b/docs/images/streams-architecture-states.jpg differ diff --git a/docs/images/streams-architecture-tasks.jpg b/docs/images/streams-architecture-tasks.jpg new file mode 100644 index 0000000000000..2e957f9763f05 Binary files /dev/null and b/docs/images/streams-architecture-tasks.jpg differ diff --git a/docs/images/streams-architecture-threads.jpg b/docs/images/streams-architecture-threads.jpg new file mode 100644 index 0000000000000..d5f10dbe69da6 Binary files /dev/null and b/docs/images/streams-architecture-threads.jpg differ diff --git a/docs/images/streams-architecture-topology.jpg b/docs/images/streams-architecture-topology.jpg new file mode 100644 index 0000000000000..f42e8cddf448c Binary files /dev/null and b/docs/images/streams-architecture-topology.jpg differ diff --git a/docs/images/streams-concepts-topology.jpg b/docs/images/streams-concepts-topology.jpg new file mode 100644 index 0000000000000..832f6d43a4235 Binary files /dev/null and b/docs/images/streams-concepts-topology.jpg differ diff --git a/docs/images/streams-table-duality-01.png b/docs/images/streams-table-duality-01.png new file mode 100644 index 0000000000000..4fa4d1bf8e474 Binary files /dev/null and b/docs/images/streams-table-duality-01.png differ diff --git a/docs/images/streams-table-duality-02.png b/docs/images/streams-table-duality-02.png new file mode 100644 index 0000000000000..4e805c10ff56c Binary files /dev/null and b/docs/images/streams-table-duality-02.png differ diff --git a/docs/images/streams-table-duality-03.png b/docs/images/streams-table-duality-03.png new file mode 100644 index 0000000000000..b0b04f59176f2 Binary files /dev/null and b/docs/images/streams-table-duality-03.png differ diff --git a/docs/images/streams-table-updates-01.png b/docs/images/streams-table-updates-01.png new file mode 100644 index 0000000000000..3a2c35ef3b90d Binary files /dev/null and b/docs/images/streams-table-updates-01.png differ diff --git a/docs/images/streams-table-updates-02.png b/docs/images/streams-table-updates-02.png new file mode 100644 index 0000000000000..a0a5b1ff53fc2 Binary files /dev/null and b/docs/images/streams-table-updates-02.png differ diff --git a/docs/introduction.html b/docs/introduction.html index 7ff62f9099bc2..556aa02eedf6f 100644 --- a/docs/introduction.html +++ b/docs/introduction.html @@ -18,7 +18,7 @@ - - -
    - -
    -
    -
    -
    - - - +
    diff --git a/docs/js/templateData.js b/docs/js/templateData.js index 40c5da195b471..b4aedf56abfad 100644 --- a/docs/js/templateData.js +++ b/docs/js/templateData.js @@ -17,5 +17,6 @@ limitations under the License. // Define variables for doc templates var context={ - "version": "0101" + "version": "0102", + "dotVersion": "0.10.2" }; \ No newline at end of file diff --git a/docs/migration.html b/docs/migration.html index db5fe60432d46..08a62715d675a 100644 --- a/docs/migration.html +++ b/docs/migration.html @@ -15,7 +15,7 @@ limitations under the License. --> - +

    Migrating from 0.7.x to 0.8

    0.8 is our first (and hopefully last) release with a non-backwards-compatible wire protocol, ZooKeeper layout, and on-disk data format. This was a chance for us to clean up a lot of cruft and start fresh. This means performing a no-downtime upgrade is more painful than normal—you cannot just swap in the new code in-place. @@ -31,4 +31,4 @@

    Migration Steps

  • Drink. - + diff --git a/docs/ops.html b/docs/ops.html index b500d6936f766..9232f651ecae1 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -14,8 +14,8 @@ See the License for the specific language governing permissions and limitations under the License. --> - - + diff --git a/docs/quickstart.html b/docs/quickstart.html index 574d7461086dc..51d85fd1b6cd3 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -15,6 +15,9 @@ limitations under the License. --> + + + + +
    diff --git a/docs/security.html b/docs/security.html index 350f0cf620700..3443c0c8fca37 100644 --- a/docs/security.html +++ b/docs/security.html @@ -19,8 +19,12 @@

    7.1 Security Overview

    In release 0.9.0.0, the Kafka community added a number of features that, used either separately or together, increases security in a Kafka cluster. The following security measures are currently supported:
      -
    1. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL (Kerberos). - SASL/PLAIN can also be used from release 0.10.0.0 onwards.
    2. +
    3. Authentication of connections to brokers from clients (producers and consumers), other brokers and tools, using either SSL or SASL. Kafka supports the following SASL mechanisms: +
        +
      • SASL/GSSAPI (Kerberos) - starting at version 0.9.0.0
      • +
      • SASL/PLAIN - starting at version 0.10.0.0
      • +
      • SASL/SCRAM-SHA-256 and SASL/SCRAM-SHA-512 - starting at version 0.10.2.0
      • +
    4. Authentication of connections from brokers to ZooKeeper
    5. Encryption of data transferred between brokers and clients, between brokers, or between brokers and tools using SSL (Note that there is a performance degradation when SSL is enabled, the magnitude of which depends on the CPU type and the JVM implementation.)
    6. Authorization of read / write operations by clients
    7. @@ -211,75 +215,125 @@

      7.2 Encryption and Authentication

      7.3 Authentication using SASL

        -
      1. SASL configuration for Kafka brokers

        -
          -
        1. Select one or more supported mechanisms to enable in the broker. GSSAPI - and PLAIN are the mechanisms currently supported in Kafka.
        2. -
        3. Add a JAAS config file for the selected mechanisms as described in the examples - for setting up GSSAPI (Kerberos) - or PLAIN.
        4. -
        5. Pass the JAAS config file location as JVM parameter to each Kafka broker. - For example: -
              -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
        6. -
        7. Configure a SASL port in server.properties, by adding at least one of - SASL_PLAINTEXT or SASL_SSL to the listeners parameter, which - contains one or more comma-separated values: -
              listeners=SASL_PLAINTEXT://host.name:port
          - If SASL_SSL is used, then SSL must also be - configured. If you are only configuring a SASL port (or if you want - the Kafka brokers to authenticate each other using SASL) then make sure - you set the same SASL protocol for inter-broker communication: -
              security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
        8. -
        9. Enable one or more SASL mechanisms in server.properties: -
              sasl.enabled.mechanisms=GSSAPI (,PLAIN)
        10. -
        11. Configure the SASL mechanism for inter-broker communication in server.properties - if using SASL for inter-broker communication: -
              sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)
        12. -
        13. Follow the steps in GSSAPI (Kerberos) - or PLAIN to configure SASL - for the enabled mechanisms. To enable multiple mechanisms in the broker, follow - the steps here.
        14. - Important notes: +
        15. JAAS configuration

          +

          Kafka uses the Java Authentication and Authorization Service + (JAAS) + for SASL configuration.

            -
          1. KafkaServer is the section name in the JAAS file used by each +
          2. JAAS configuration for Kafka brokers
            + +

            KafkaServer is the section name in the JAAS file used by each KafkaServer/Broker. This section provides SASL configuration options for the broker including any SASL client connections made by the broker - for inter-broker communication.

          3. -
          4. Client section is used to authenticate a SASL connection with + for inter-broker communication.

            + +

            Client section is used to authenticate a SASL connection with zookeeper. It also allows the brokers to set SASL ACL on zookeeper nodes which locks these nodes down so that only the brokers can modify it. It is necessary to have the same principal name across all brokers. If you want to use a section name other than Client, set the system property zookeeper.sasl.client to the appropriate - name (e.g., -Dzookeeper.sasl.client=ZkClient).

          5. -
          6. ZooKeeper uses "zookeeper" as the service name by default. If you + name (e.g., -Dzookeeper.sasl.client=ZkClient).

            + +

            ZooKeeper uses "zookeeper" as the service name by default. If you want to change this, set the system property zookeeper.sasl.client.username to the appropriate name - (e.g., -Dzookeeper.sasl.client.username=zk).

          7. -
          + (e.g., -Dzookeeper.sasl.client.username=zk).

        16. + +
        17. JAAS configuration for Kafka clients
          + +

          Clients may configure JAAS using the client configuration property + sasl.jaas.config + or using the static JAAS config file + similar to brokers.

          + +
            +
          1. JAAS configuration using client configuration property
            +

            Clients may specify JAAS configuration as a producer or consumer property without + creating a physical configuration file. This mode also enables different producers + and consumers within the same JVM to use different credentials by specifying + different properties for each client. If both static JAAS configuration system property + java.security.auth.login.config and client property sasl.jaas.config + are specified, the client property will be used.

            + +

            See GSSAPI (Kerberos), + PLAIN or + SCRAM for example configurations.

          2. + +
          3. JAAS configuration using static config file
            + To configure SASL authentication on the clients using static JAAS config file: +
              +
            1. Add a JAAS config file with a client login section named KafkaClient. Configure + a login module in KafkaClient for the selected mechanism as described in the examples + for setting up GSSAPI (Kerberos), + PLAIN or + SCRAM. + For example, GSSAPI + credentials may be configured as: +
              +        KafkaClient {
              +        com.sun.security.auth.module.Krb5LoginModule required
              +        useKeyTab=true
              +        storeKey=true
              +        keyTab="/etc/security/keytabs/kafka_client.keytab"
              +        principal="kafka-client-1@EXAMPLE.COM";
              +    };
              +
            2. +
            3. Pass the JAAS config file location as JVM parameter to each client JVM. For example: +
                  -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
            4. +
            +
          4. +
          +
      2. -
      3. SASL configuration for Kafka clients

        - SASL authentication is only supported for the new Java Kafka producer and - consumer, the older API is not supported. To configure SASL authentication - on the clients: +
      4. SASL configuration

        + +

        SASL may be used with PLAINTEXT or SSL as the transport layer using the + security protocol SASL_PLAINTEXT or SASL_SSL respectively. If SASL_SSL is + used, then SSL must also be configured.

        +
          -
        1. Select a SASL mechanism for authentication.
        2. -
        3. Add a JAAS config file for the selected mechanism as described in the examples - for setting up GSSAPI (Kerberos) - or PLAIN. KafkaClient is the - section name in the JAAS file used by Kafka clients.
        4. -
        5. Pass the JAAS config file location as JVM parameter to each client JVM. For example: -
              -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
        6. -
        7. Configure the following properties in producer.properties or - consumer.properties: -
              security.protocol=SASL_PLAINTEXT (or SASL_SSL)
          -        sasl.mechanism=GSSAPI (or PLAIN)
        8. -
        9. Follow the steps in GSSAPI (Kerberos) - or PLAIN to configure SASL - for the selected mechanism.
        10. +
        11. SASL mechanisms
          + Kafka supports the following SASL mechanisms: + +
        12. +
        13. SASL configuration for Kafka brokers
          +
            +
          1. Configure a SASL port in server.properties, by adding at least one of + SASL_PLAINTEXT or SASL_SSL to the listeners parameter, which + contains one or more comma-separated values: +
                listeners=SASL_PLAINTEXT://host.name:port
            + If you are only configuring a SASL port (or if you want + the Kafka brokers to authenticate each other using SASL) then make sure + you set the same SASL protocol for inter-broker communication: +
                security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
          2. +
          3. Select one or more supported mechanisms + to enable in the broker and follow the steps to configure SASL for the mechanism. + To enable multiple mechanisms in the broker, follow the steps + here.
          4. +
          +
        14. +
        15. SASL configuration for Kafka clients
          +

          SASL authentication is only supported for the new Java Kafka producer and + consumer, the older API is not supported.

          + +

          To configure SASL authentication on the clients, select a SASL + mechanism that is enabled in + the broker for client authentication and follow the steps to configure SASL + for the selected mechanism.

      5. Authentication using SASL/Kerberos

        @@ -320,7 +374,7 @@

        7.3 Authentication using SASL KafkaServer section in the JAAS file tells the broker which principal to use and the location of the keytab where this principal is stored. It allows the broker to login using the keytab specified in this section. See notes for more details on Zookeeper SASL configuration. -
      6. Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see here for more details): +
      7. Pass the JAAS and optionally the krb5 file locations as JVM parameters to each Kafka broker (see here for more details):
            -Djava.security.krb5.conf=/etc/kafka/krb5.conf
                 -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
      8. @@ -331,7 +385,7 @@

        7.3 Authentication using SASL -

      9. We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so: + We must also configure the service name in server.properties, which should match the principal name of the kafka brokers. In the above example, principal is "kafka/kafka1.hostname.com@EXAMPLE.com", so:
            sasl.kerberos.service.name=kafka
      @@ -339,37 +393,43 @@

      7.3 Authentication using SASL
    8. - Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their own principal (usually with the same name as the user running the client), so obtain or create these principals as needed. Then create a JAAS file for each principal. - The KafkaClient section describes how the clients like producer and consumer can connect to the Kafka Broker. The following is an example configuration for a client using a keytab (recommended for long-running processes): + Clients (producers, consumers, connect workers, etc) will authenticate to the cluster with their + own principal (usually with the same name as the user running the client), so obtain or create + these principals as needed. Then configure the JAAS configuration property for each client. + Different clients within a JVM may run as different users by specifiying different principals. + The property sasl.jaas.config in producer.properties or consumer.properties describes + how clients like producer and consumer can connect to the Kafka Broker. The following is an example + configuration for a client using a keytab (recommended for long-running processes):
      -        KafkaClient {
      -            com.sun.security.auth.module.Krb5LoginModule required
      -            useKeyTab=true
      -            storeKey=true
      -            keyTab="/etc/security/keytabs/kafka_client.keytab"
      -            principal="kafka-client-1@EXAMPLE.COM";
      -        };
      - - For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used along with "useTicketCache=true" as in: + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useKeyTab=true \ + storeKey=true \ + keyTab="/etc/security/keytabs/kafka_client.keytab" \ + principal="kafka-client-1@EXAMPLE.COM"; + + For command-line utilities like kafka-console-consumer or kafka-console-producer, kinit can be used + along with "useTicketCache=true" as in:
      -        KafkaClient {
      -            com.sun.security.auth.module.Krb5LoginModule required
      -            useTicketCache=true;
      -        };
      + sasl.jaas.config=com.sun.security.auth.module.Krb5LoginModule required \ + useTicketCache=true; + + JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.
    9. +
    10. Make sure the keytabs configured in the JAAS configuration are readable by the operating system user who is starting kafka client.
    11. -
    12. Pass the JAAS and optionally krb5 file locations as JVM parameters to each client JVM (see here for more details): -
          -Djava.security.krb5.conf=/etc/kafka/krb5.conf
      -        -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
    13. -
    14. Make sure the keytabs configured in the kafka_client_jaas.conf are readable by the operating system user who is starting kafka client.
    15. -
    16. Configure the following properties in producer.properties or consumer.properties: -
          security.protocol=SASL_PLAINTEXT (or SASL_SSL)
      -        sasl.mechanism=GSSAPI
      -        sasl.kerberos.service.name=kafka
    17. +
    18. Optionally pass the krb5 file locations as JVM parameters to each client JVM (see here for more details): +
          -Djava.security.krb5.conf=/etc/kafka/krb5.conf
    19. +
    20. Configure the following properties in producer.properties or consumer.properties: +
      +    security.protocol=SASL_PLAINTEXT (or SASL_SSL)
      +    sasl.mechanism=GSSAPI
      +    sasl.kerberos.service.name=kafka
  • - +
  • Authentication using SASL/PLAIN

    SASL/PLAIN is a simple username/password authentication mechanism that is typically used with TLS for encryption to implement secure authentication. Kafka supports a default implementation for SASL/PLAIN which can be extended for production use as described here.

    @@ -404,22 +464,25 @@

    7.3 Authentication using SASL

    Configuring Kafka Clients
    To configure SASL authentication on the clients:
      -
    1. The KafkaClient section describes how the clients like producer and consumer can connect to the Kafka Broker. - The following is an example configuration for a client for the PLAIN mechanism: +
    2. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. + The login module describes how the clients like producer and consumer can connect to the Kafka Broker. + The following is an example configuration for a client for the PLAIN mechanism:
      -        KafkaClient {
      -            org.apache.kafka.common.security.plain.PlainLoginModule required
      -            username="alice"
      -            password="alice-secret";
      -        };
      - The properties username and password in the KafkaClient section are used by clients to configure + sasl.jaas.config=org.apache.kafka.common.security.plain.PlainLoginModule required \ + username="alice" \ + password="alice-secret"; +

      The options username and password are used by clients to configure the user for client connections. In this example, clients connect to the broker as user alice. -

    3. -
    4. Pass the JAAS config file location as JVM parameter to each client JVM: -
          -Djava.security.auth.login.config=/etc/kafka/kafka_client_jaas.conf
    5. + Different clients within a JVM may connect as different users by specifying different user names + and passwords in sasl.jaas.config.

      + +

      JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.

    6. Configure the following properties in producer.properties or consumer.properties: -
          security.protocol=SASL_SSL
      -        sasl.mechanism=PLAIN
    7. +
      +    security.protocol=SASL_SSL
      +    sasl.mechanism=PLAIN
  • Use of SASL/PLAIN in production
    @@ -451,6 +514,111 @@

    7.3 Authentication using SASL

  • + +
  • Authentication using SASL/SCRAM

    +

    Salted Challenge Response Authentication Mechanism (SCRAM) is a family of SASL mechanisms that + addresses the security concerns with traditional mechanisms that perform username/password authentication + like PLAIN and DIGEST-MD5. The mechanism is defined in RFC 5802. + Kafka supports SCRAM-SHA-256 and SCRAM-SHA-512 which + can be used with TLS to perform secure authentication. The username is used as the authenticated + Principal for configuration of ACLs etc. The default SCRAM implementation in Kafka + stores SCRAM credentials in Zookeeper and is suitable for use in Kafka installations where Zookeeper + is on a private network. Refer to Security Considerations + for more details.

    +
      +
    1. Creating SCRAM Credentials
      +

      The SCRAM implementation in Kafka uses Zookeeper as credential store. Credentials can be created in + Zookeeper using kafka-configs.sh. For each SCRAM mechanism enabled, credentials must be created + by adding a config with the mechanism name. Credentials for inter-broker communication must be created + before Kafka brokers are started. Client credentials may be created and updated dynamically and updated + credentials will be used to authenticate new connections.

      +

      Create SCRAM credentials for user alice with password alice-secret: +

      +    bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[iterations=8192,password=alice-secret],SCRAM-SHA-512=[password=alice-secret]' --entity-type users --entity-name alice
      +        
      +

      The default iteration count of 4096 is used if iterations are not specified. A random salt is created + and the SCRAM identity consisting of salt, iterations, StoredKey and ServerKey are stored in Zookeeper. + See RFC 5802 for details on SCRAM identity and the individual fields. +

      The following examples also require a user admin for inter-broker communication which can be created using: +

      +    bin/kafka-configs.sh --zookeeper localhost:2181 --alter --add-config 'SCRAM-SHA-256=[password=admin-secret],SCRAM-SHA-512=[password=admin-secret]' --entity-type users --entity-name admin
      +        
      +

      Existing credentials may be listed using the --describe option: +

      +   bin/kafka-configs.sh --zookeeper localhost:2181 --describe --entity-type users --entity-name alice
      +        
      +

      Credentials may be deleted for one or more SCRAM mechanisms using the --delete option: +

      +   bin/kafka-configs.sh --zookeeper localhost:2181 --alter --delete-config 'SCRAM-SHA-512' --entity-type users --entity-name alice
      +        
      +
    2. +
    3. Configuring Kafka Brokers
      +
        +
      1. Add a suitably modified JAAS file similar to the one below to each Kafka broker's config directory, let's call it kafka_server_jaas.conf for this example: +
        +    KafkaServer {
        +        org.apache.kafka.common.security.scram.ScramLoginModule required
        +        username="admin"
        +        password="admin-secret"
        +    };
        + The properties username and password in the KafkaServer section are used by + the broker to initiate connections to other brokers. In this example, admin is the user for + inter-broker communication.
      2. +
      3. Pass the JAAS config file location as JVM parameter to each Kafka broker: +
            -Djava.security.auth.login.config=/etc/kafka/kafka_server_jaas.conf
      4. +
      5. Configure SASL port and SASL mechanisms in server.properties as described here. For example: +
        +    listeners=SASL_SSL://host.name:port
        +    security.inter.broker.protocol=SASL_SSL
        +    sasl.mechanism.inter.broker.protocol=SCRAM-SHA-256 (or SCRAM-SHA-512)
        +    sasl.enabled.mechanisms=SCRAM-SHA-256 (or SCRAM-SHA-512)
      6. +
      +
    4. + +
    5. Configuring Kafka Clients
      + To configure SASL authentication on the clients: +
        +
      1. Configure the JAAS configuration property for each client in producer.properties or consumer.properties. + The login module describes how the clients like producer and consumer can connect to the Kafka Broker. + The following is an example configuration for a client for the SCRAM mechanisms: +
        +   sasl.jaas.config=org.apache.kafka.common.security.scram.ScramLoginModule required \
        +        username="alice" \
        +        password="alice-secret";
        + +

        The options username and password are used by clients to configure + the user for client connections. In this example, clients connect to the broker as user alice. + Different clients within a JVM may connect as different users by specifying different user names + and passwords in sasl.jaas.config.

        + +

        JAAS configuration for clients may alternatively be specified as a JVM parameter similar to brokers + as described here. Clients use the login section named + KafkaClient. This option allows only one user for all client connections from a JVM.

      2. + +
      3. Configure the following properties in producer.properties or consumer.properties: +
        +    security.protocol=SASL_SSL
        +    sasl.mechanism=SCRAM-SHA-256 (or SCRAM-SHA-512)
      4. +
      +
    6. +
    7. Security Considerations for SASL/SCRAM
      +
        +
      • The default implementation of SASL/SCRAM in Kafka stores SCRAM credentials in Zookeeper. This + is suitable for production use in installations where Zookeeper is secure and on a private network.
      • +
      • Kafka supports only the strong hash functions SHA-256 and SHA-512 with a minimum iteration count + of 4096. Strong hash functions combined with strong passwords and high iteration counts protect + against brute force attacks if Zookeeper security is compromised.
      • +
      • SCRAM should be used only with TLS-encryption to prevent interception of SCRAM exchanges. This + protects against dictionary or brute force attacks and against impersonation if Zookeeper is compromised.
      • +
      • The default SASL/SCRAM implementation may be overridden using custom login modules in installations + where Zookeeper is not secure. See here for details.
      • +
      • For more details on security considerations, refer to + RFC 5802. +
      +
    8. +
    +
  • +
  • Enabling multiple SASL mechanisms in a broker

    1. Specify configuration for the login modules of all enabled mechanisms in the KafkaServer section of the JAAS config file. For example: @@ -468,12 +636,14 @@

      7.3 Authentication using SASL

    2. -
    3. Enable the SASL mechanisms in server.properties:
          sasl.enabled.mechanisms=GSSAPI,PLAIN
    4. +
    5. Enable the SASL mechanisms in server.properties:
          sasl.enabled.mechanisms=GSSAPI,PLAIN,SCRAM-SHA-256,SCRAM-SHA-512
    6. Specify the SASL security protocol and mechanism for inter-broker communication in server.properties if required: -
          security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
      -        sasl.mechanism.inter.broker.protocol=GSSAPI (or PLAIN)
    7. -
    8. Follow the mechanism-specific steps in GSSAPI (Kerberos) - and PLAIN to configure SASL for the enabled mechanisms.
    9. +
      +    security.inter.broker.protocol=SASL_PLAINTEXT (or SASL_SSL)
      +    sasl.mechanism.inter.broker.protocol=GSSAPI (or one of the other enabled mechanisms)
      +
    10. Follow the mechanism-specific steps in GSSAPI (Kerberos), + PLAIN and SCRAM + to configure SASL for the enabled mechanisms.
  • Modifying SASL mechanism in a Running Cluster

    @@ -733,7 +903,7 @@

    7.6.2 Migrating cluste Here is an example of how to run the migration tool:
    -    ./bin/zookeeper-security-migration --zookeeper.acl=secure --zookeeper.connection=localhost:2181
    +    ./bin/zookeeper-security-migration --zookeeper.acl=secure --zookeeper.connect=localhost:2181
         

    Run this to see the full list of parameters:

    diff --git a/docs/streams.html b/docs/streams.html
    index dec17efa2d270..fe0e84ee3b74c 100644
    --- a/docs/streams.html
    +++ b/docs/streams.html
    @@ -1,19 +1,19 @@
    -
    +
     
     
     
    @@ -25,19 +25,30 @@ 

    Streams

    Overview

  • - Developer guide + Core Concepts +
  • +
  • + Architecture +
  • +
  • + Developer Guide
  • +
  • + Upgrade Guide and API Changes +
  • -

    Overview

    +

    Overview

    Kafka Streams is a client library for processing and analyzing data stored in Kafka and either write the resulting data back to Kafka or send the final output to an external system. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management of application state. +

    +

    Kafka Streams has a low barrier to entry: You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model.

    @@ -47,192 +58,353 @@

    Overview

    • Designed as a simple and lightweight client library, which can be easily embedded in any Java application and integrated with any existing packaging, deployment and operational tools that users have for their streaming applications.
    • 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 joins and windowed aggregations.
    • -
    • Employs one-record-at-a-time processing to achieve low processing latency, and supports event-time based windowing operations.
    • +
    • Supports fault-tolerant local state, which enables very fast and efficient stateful operations like windowed joins and aggregations.
    • +
    • Employs one-record-at-a-time processing to achieve millisecond processing latency, and supports event-time based windowing operations with late arrival of records.
    • Offers necessary stream processing primitives, along with a high-level Streams DSL and a low-level Processor API.
    +
    -

    Developer Guide

    +

    Core Concepts

    - There is a quickstart example that provides how to run a stream processing program coded in the Kafka Streams library. - This section focuses on how to write, configure, and execute a Kafka Streams application. + We first summarize the key concepts of Kafka Streams.

    -

    Core Concepts

    +

    Stream Processing Topology

    -

    - We first summarize the key concepts of Kafka Streams. -

    +
      +
    • A stream is the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a data record is defined as a key-value pair.
    • +
    • A stream processing application is any program that makes use of the Kafka Streams library. It defines its computational logic through one or more processor topologies, where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges).
    • +
    • A stream processor is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently produce one or more output records to its downstream processors.
    • +
    -
    Stream Processing Topology
    + There are two special processors in the topology:
      -
    • A stream is the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a data record is defined as a key-value pair.
    • -
    • A stream processing application written in Kafka Streams defines its computational logic through one or more processor topologies, where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges).
    • -
    • A stream processor is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently producing one or more output records to its downstream processors.
    • +
    • Source Processor: A source processor is a special type of stream processor that does not have any upstream processors. It produces an input stream to its topology from one or multiple Kafka topics by consuming records from these topics and forward them to its down-stream processors.
    • +
    • Sink Processor: A sink processor is a special type of stream processor that does not have down-stream processors. It sends any received records from its up-stream processors to a specified Kafka topic.
    + + +

    + Kafka Streams offers two ways to define the stream processing topology: the Kafka Streams DSL provides + the most common data transformation operations such as map, filter, join and aggregations out of the box; the lower-level Processor API allows + developers define and connect custom processors as well as to interact with state stores. +

    +

    - Kafka Streams offers two ways to define the stream processing topology: the Kafka Streams DSL provides - the most common data transformation operations such as map and filter; the lower-level Processor API allows - developers define and connect custom processors as well as to interact with state stores. + A processor topology is merely a logical abstraction for your stream processing code. + At runtime, the logical topology is instantiated and replicated inside the application for parallel processing (see Stream Partitions and Tasks for details).

    -
    Time
    +

    Time

    - A critical aspect in stream processing is the notion of time, and how it is modeled and integrated. - For example, some operations such as windowing are defined based on time boundaries. + A critical aspect in stream processing is the notion of time, and how it is modeled and integrated. + For example, some operations such as windowing are defined based on time boundaries.

    - Common notions of time in streams are: + Common notions of time in streams are:

      -
    • Event time - The point in time when an event or data record occurred, i.e. was originally created "at the source".
    • -
    • Processing time - The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time.
    • -
    • Ingestion time - The point in time when an event or data record is stored in a topic partition by a Kafka broker. The difference to event time is that this ingestion timestamp is generated when the record is appended to the target topic by the Kafka broker, not when the record is created "at the source". The difference to processing time is that processing time is when the stream processing application processes the record. For example, if a record is never processed, there is no notion of processing time for it, but it still has an ingestion time. +
    • Event time - The point in time when an event or data record occurred, i.e. was originally created "at the source". Example: If the event is a geo-location change reported by a GPS sensor in a car, then the associated event-time would be the time when the GPS sensor captured the location change.
    • +
    • Processing time - The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time. Example: Imagine an analytics application that reads and processes the geo-location data reported from car sensors to present it to a fleet management dashboard. Here, processing-time in the analytics application might be milliseconds or seconds (e.g. for real-time pipelines based on Apache Kafka and Kafka Streams) or hours (e.g. for batch pipelines based on Apache Hadoop or Apache Spark) after event-time.
    • +
    • Ingestion time - The point in time when an event or data record is stored in a topic partition by a Kafka broker. The difference to event time is that this ingestion timestamp is generated when the record is appended to the target topic by the Kafka broker, not when the record is created "at the source". The difference to processing time is that processing time is when the stream processing application processes the record. For example, if a record is never processed, there is no notion of processing time for it, but it still has an ingestion time.

    - The choice between event-time and ingestion-time is actually done through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x onwards, timestamps are automatically embedded into Kafka messages. Depending on Kafka's configuration these timestamps represent event-time or ingestion-time. The respective Kafka configuration setting can be specified on the broker level or per topic. The default timestamp extractor in Kafka Streams will retrieve these embedded timestamps as-is. Hence, the effective time semantics of your application depend on the effective Kafka configuration for these embedded timestamps. + The choice between event-time and ingestion-time is actually done through the configuration of Kafka (not Kafka Streams): From Kafka 0.10.x onwards, timestamps are automatically embedded into Kafka messages. Depending on Kafka's configuration these timestamps represent event-time or ingestion-time. The respective Kafka configuration setting can be specified on the broker level or per topic. The default timestamp extractor in Kafka Streams will retrieve these embedded timestamps as-is. Hence, the effective time semantics of your application depend on the effective Kafka configuration for these embedded timestamps.

    - Kafka Streams assigns a timestamp to every data record - via the TimestampExtractor interface. - Concrete implementations of this interface may retrieve or compute timestamps based on the actual contents of data records such as an embedded timestamp field - to provide event-time semantics, or use any other approach such as returning the current wall-clock time at the time of processing, - thereby yielding processing-time semantics to stream processing applications. - Developers can thus enforce different notions of time depending on their business needs. For example, - per-record timestamps describe the progress of a stream with regards to time (although records may be out-of-order within the stream) and - are leveraged by time-dependent operations such as joins. + Kafka Streams assigns a timestamp to every data record + via the TimestampExtractor interface. + Concrete implementations of this interface may retrieve or compute timestamps based on the actual contents of data records such as an embedded timestamp field + to provide event-time semantics, or use any other approach such as returning the current wall-clock time at the time of processing, + thereby yielding processing-time semantics to stream processing applications. + Developers can thus enforce different notions of time depending on their business needs. For example, + per-record timestamps describe the progress of a stream with regards to time (although records may be out-of-order within the stream) and + are leveraged by time-dependent operations such as joins.

    - Finally, whenever a Kafka Streams application writes records to Kafka, then it will also assign timestamps to these new records. The way the timestamps are assigned depends on the context: + Finally, whenever a Kafka Streams application writes records to Kafka, then it will also assign timestamps to these new records. The way the timestamps are assigned depends on the context: +

    +
    • 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 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.
    + +

    States

    + +

    + Some stream processing applications don't require state, which means the processing of a message is independent from + the processing of all other messages. + However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you + can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL. +

    +

    + Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data. + This is an important capability when implementing stateful operations. + Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. + These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. + Kafka Streams offers fault-tolerance and automatic recovery for local state stores.

    +

    + Kafka Streams allows direct read-only queries of the state stores by methods, threads, processes or applications external to the stream processing application that created the state stores. This is provided through a feature called Interactive Queries. All stores are named and Interactive Queries exposes only the read operations of the underlying implementation. +

    +
    -
    States
    +

    Architecture

    + + Kafka Streams simplifies application development by building on the Kafka producer and consumer libraries and leveraging the native capabilities of + Kafka to offer data parallelism, distributed coordination, fault tolerance, and operational simplicity. In this section, we describe how Kafka Streams works underneath the covers.

    - Some stream processing applications don't require state, which means the processing of a message is independent from - the processing of all other messages. - However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you - can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL. + The picture below shows the anatomy of an application that uses the Kafka Streams library. Let's walk through some details.

    + + +

    Stream Partitions and Tasks

    +

    - Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data. - This is an important capability when implementing stateful operations. - Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. - These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. - Kafka Streams offers fault-tolerance and automatic recovery for local state stores. + The messaging layer of Kafka partitions data for storing and transporting it. Kafka Streams partitions data for processing it. + In both cases, this partitioning is what enables data locality, elasticity, scalability, high performance, and fault tolerance. + Kafka Streams uses the concepts of partitions and tasks as logical units of its parallelism model based on Kafka topic partitions. + There are close links between Kafka Streams and Kafka in the context of parallelism:

    + +
      +
    • Each stream partition is a totally ordered sequence of data records and maps to a Kafka topic partition.
    • +
    • A data record in the stream maps to a Kafka message from that topic.
    • +
    • The keys of data records determine the partitioning of data in both Kafka and Kafka Streams, i.e., how data is routed to specific partitions within topics.
    • +
    +

    - Kafka Streams allows direct read-only queries of the state stores by methods, threads, processes or applications external to the stream processing application that created the state stores. This is provided through a feature called Interactive Queries. All stores are named and Interactive Queries exposes only the read operations of the underlying implementation. + An application's processor topology is scaled by breaking it into multiple tasks. + More specifically, Kafka Streams creates a fixed number of tasks based on the input stream partitions for the application, + with each task assigned a list of partitions from the input streams (i.e., Kafka topics). The assignment of partitions to tasks + never changes so that each task is a fixed unit of parallelism of the application. Tasks can then instantiate their own processor topology + based on the assigned partitions; they also maintain a buffer for each of its assigned partitions and process messages one-at-a-time from + these record buffers. As a result stream tasks can be processed independently and in parallel without manual intervention.

    + +

    + It is important to understand that Kafka Streams is not a resource manager, but a library that “runs” anywhere its stream processing application runs. + Multiple instances of the application are executed either on the same machine, or spread across multiple machines and tasks can be distributed automatically + by the library to those running application instances. The assignment of partitions to tasks never changes; if an application instance fails, all its assigned + tasks will be automatically restarted on other instances and continue to consume from the same stream partitions. +

    + +

    + The following diagram shows two tasks each assigned with one partition of the input streams. +

    +
    + +

    Threading Model

    +

    - As we have mentioned above, the computational logic of a Kafka Streams application is defined as a processor topology. - Currently Kafka Streams provides two sets of APIs to define the processor topology, which will be described in the subsequent sections. + Kafka Streams allows the user to configure the number of threads that the library can use to parallelize processing within an application instance. + Each thread can execute one or more tasks with their processor topologies independently. For example, the following diagram shows one stream thread running two stream tasks. +

    + + +

    + Starting more stream threads or more instances of the application merely amounts to replicating the topology and having it process a different subset of Kafka partitions, effectively parallelizing processing. + It is worth noting that there is no shared state amongst the threads, so no inter-thread coordination is necessary. This makes it very simple to run topologies in parallel across the application instances and threads. + The assignment of Kafka topic partitions amongst the various stream threads is transparently handled by Kafka Streams leveraging Kafka's coordination functionality.

    -

    Low-Level Processor API

    +

    + As we described above, scaling your stream processing application with Kafka Streams is easy: you merely need to start additional instances of your application, + and Kafka Streams takes care of distributing partitions amongst tasks that run in the application instances. You can start as many threads of the application + as there are input Kafka topic partitions so that, across all running instances of an application, every thread (or rather, the tasks it runs) has at least one input partition to process. +

    +
    -
    Processor
    +

    Local State Stores

    - Developers can define their customized processing logic by implementing the Processor interface, which - provides process and punctuate methods. The process method is performed on each - of the received record; and the punctuate method is performed periodically based on elapsed time. - In addition, the processor can maintain the current ProcessorContext instance variable initialized in the - init method, and use the context to schedule the punctuation period (context().schedule), to - forward the modified / new key-value pair to downstream processors (context().forward), to commit the current - processing progress (context().commit), etc. + Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data, + which is an important capability when implementing stateful operations. The Kafka Streams DSL, for example, automatically creates + and manages such state stores when you are calling stateful operators such as join() or aggregate(), or when you are windowing a stream. +

    + +

    + Every stream task in a Kafka Streams application may embed one or more local state stores that can be accessed via APIs to store and query data required for processing. + Kafka Streams offers fault-tolerance and automatic recovery for such local state stores.

    -
    -            public class MyProcessor extends Processor {
    -                private ProcessorContext context;
    -                private KeyValueStore kvStore;
    +        

    + The following diagram shows two stream tasks with their dedicated local state stores. +

    + +
    - @Override - @SuppressWarnings("unchecked") - public void init(ProcessorContext context) { - this.context = context; - this.context.schedule(1000); - this.kvStore = (KeyValueStore) context.getStateStore("Counts"); - } +

    Fault Tolerance

    - @Override - public void process(String dummy, String line) { - String[] words = line.toLowerCase().split(" "); +

    + Kafka Streams builds on fault-tolerance capabilities integrated natively within Kafka. Kafka partitions are highly available and replicated; so when stream data is persisted to Kafka it is available + even if the application fails and needs to re-process it. Tasks in Kafka Streams leverage the fault-tolerance capability + offered by the Kafka consumer client to handle failures. + If a task runs on a machine that fails, Kafka Streams automatically restarts the task in one of the remaining running instances of the application. +

    - for (String word : words) { - Integer oldValue = this.kvStore.get(word); +

    + In addition, Kafka Streams makes sure that the local state stores are robust to failures, too. For each state store, it maintains a replicated changelog Kafka topic in which it tracks any state updates. + These changelog topics are partitioned as well so that each local state store instance, and hence the task accessing the store, has its own dedicated changelog topic partition. + Log compaction is enabled on the changelog topics so that old data can be purged safely to prevent the topics from growing indefinitely. + If tasks run on a machine that fails and are restarted on another machine, Kafka Streams guarantees to restore their associated state stores to the content before the failure by + replaying the corresponding changelog topics prior to resuming the processing on the newly started tasks. As a result, failure handling is completely transparent to the end user. +

    - if (oldValue == null) { - this.kvStore.put(word, 1); - } else { - this.kvStore.put(word, oldValue + 1); - } - } - } +

    + Note that the cost of task (re)initialization typically depends primarily on the time for restoring the state by replaying the state stores' associated changelog topics. + To minimize this restoration time, users can configure their applications to have standby replicas of local states (i.e. fully replicated copies of the state). + When a task migration happens, Kafka Streams then attempts to assign a task to an application instance where such a standby replica already exists in order to minimize + the task (re)initialization cost. See num.standby.replicas at the Kafka Streams Configs Section. +

    +
    - @Override - public void punctuate(long timestamp) { - KeyValueIterator iter = this.kvStore.all(); +

    Developer Guide

    - while (iter.hasNext()) { - KeyValue entry = iter.next(); - context.forward(entry.key, entry.value.toString()); - } +

    + There is a quickstart example that provides how to run a stream processing program coded in the Kafka Streams library. + This section focuses on how to write, configure, and execute a Kafka Streams application. +

    - iter.close(); - context.commit(); - } +

    + As we have mentioned above, the computational logic of a Kafka Streams application is defined as a processor topology. + Currently Kafka Streams provides two sets of APIs to define the processor topology, which will be described in the subsequent sections. +

    + +

    Low-Level Processor API

    + +

    Processor

    + +

    + As mentioned in the Core Concepts section, a stream processor is a node in the processor topology that represents a single processing step. + With the Processor API developers can define arbitrary stream processors that process one received record at a time, and connect these processors with + their associated state stores to compose the processor topology that represents their customized processing logic. +

    - @Override - public void close() { - this.kvStore.close(); - } - }; -
    +

    + The Processor interface provides two main API methods: + process and punctuate. The process method is performed on each + of the received record; and the punctuate method is performed periodically based on elapsed time. + In addition, the processor can maintain the current ProcessorContext instance variable initialized in the + init method, and use the context to schedule the punctuation period (context().schedule), to + forward the modified / new key-value pair to downstream processors (context().forward), to commit the current + processing progress (context().commit), etc. +

    + +

    + The following example Processor implementation defines a simple word-count algorithm: +

    + +
    +public class MyProcessor implements Processor<String, String> {
    +    private ProcessorContext context;
    +    private KeyValueStore<String, Long> kvStore;
    +
    +    @Override
    +    @SuppressWarnings("unchecked")
    +    public void init(ProcessorContext context) {
    +        // keep the processor context locally because we need it in punctuate() and commit()
    +        this.context = context;
    +
    +        // call this processor's punctuate() method every 1000 milliseconds.
    +        this.context.schedule(1000);
    +
    +        // retrieve the key-value store named "Counts"
    +        this.kvStore = (KeyValueStore<String, Long>) context.getStateStore("Counts");
    +    }
    +
    +    @Override
    +    public void process(String dummy, String line) {
    +        String[] words = line.toLowerCase().split(" ");
    +
    +        for (String word : words) {
    +            Long oldValue = this.kvStore.get(word);
    +
    +            if (oldValue == null) {
    +                this.kvStore.put(word, 1L);
    +            } else {
    +                this.kvStore.put(word, oldValue + 1L);
    +            }
    +        }
    +    }
    +
    +    @Override
    +    public void punctuate(long timestamp) {
    +        KeyValueIterator<String, Long> iter = this.kvStore.all();
    +
    +        while (iter.hasNext()) {
    +            KeyValue<String, Long> entry = iter.next();
    +            context.forward(entry.key, entry.value.toString());
    +        }
    +
    +        iter.close();
    +        // commit the current processing progress
    +        context.commit();
    +    }
    +
    +    @Override
    +    public void close() {
    +        // close any resources managed by this processor.
    +        // Note: Do not close any StateStores as these are managed
    +        // by the library
    +    }
    +};
    +

    In the above implementation, the following actions are performed: +

    • In the init method, schedule the punctuation every 1 second and retrieve the local state store by its name "Counts".
    • In the process method, upon each received record, split the value string into words, and update their counts into the state store (we will talk about this feature later in the section).
    • In the punctuate method, iterate the local state store and send the aggregated counts to the downstream processor, and commit the current stream state.
    -

    -
    Processor Topology
    + +

    Processor Topology

    With the customized processors defined in the Processor API, developers can use the TopologyBuilder to build a processor topology by connecting these processors together: +

    -
    -            TopologyBuilder builder = new TopologyBuilder();
    +
    +TopologyBuilder builder = new TopologyBuilder();
     
    -            builder.addSource("SOURCE", "src-topic")
    +builder.addSource("SOURCE", "src-topic")
    +    // add "PROCESS1" node which takes the source processor "SOURCE" as its upstream processor
    +    .addProcessor("PROCESS1", () -> new MyProcessor1(), "SOURCE")
     
    -                .addProcessor("PROCESS1", MyProcessor1::new /* the ProcessorSupplier that can generate MyProcessor1 */, "SOURCE")
    -                .addProcessor("PROCESS2", MyProcessor2::new /* the ProcessorSupplier that can generate MyProcessor2 */, "PROCESS1")
    -                .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
    +    // add "PROCESS2" node which takes "PROCESS1" as its upstream processor
    +    .addProcessor("PROCESS2", () -> new MyProcessor2(), "PROCESS1")
     
    -                .addSink("SINK1", "sink-topic1", "PROCESS1")
    -                .addSink("SINK2", "sink-topic2", "PROCESS2")
    -                .addSink("SINK3", "sink-topic3", "PROCESS3");
    -        
    + // add "PROCESS3" node which takes "PROCESS1" as its upstream processor + .addProcessor("PROCESS3", () -> new MyProcessor3(), "PROCESS1") + + // add the sink processor node "SINK1" that takes Kafka topic "sink-topic1" + // as output and the "PROCESS1" node as its upstream processor + .addSink("SINK1", "sink-topic1", "PROCESS1") + + // add the sink processor node "SINK2" that takes Kafka topic "sink-topic2" + // as output and the "PROCESS2" node as its upstream processor + .addSink("SINK2", "sink-topic2", "PROCESS2") + + // add the sink processor node "SINK3" that takes Kafka topic "sink-topic3" + // as output and the "PROCESS3" node as its upstream processor + .addSink("SINK3", "sink-topic3", "PROCESS3"); +
    There are several steps in the above code to build the topology, and here is a quick walk through: @@ -241,90 +413,184 @@
    Proces
  • Three processor nodes are then added using the addProcessor method; here the first processor is a child of the "SOURCE" node, but is the parent of the other two processors.
  • Finally three sink nodes are added to complete the topology using the addSink method, each piping from a different parent processor node and writing to a separate topic.
  • + +

    State Stores

    + +

    + Note that the Processor API is not limited to only accessing the current records as they arrive in the process() method, but can also maintain processing states + that keep recently arrived records to use in stateful processing operations such as windowed joins or aggregation. + To take advantage of these states, users can define a state store by implementing the StateStore interface (the Kafka Streams library also has a few extended interfaces such as KeyValueStore); + in practice, though, users usually do not need to customize such a state store from scratch but can simply use the Stores factory to define a state store by specifying whether it should be persistent, log-backed, etc. + In the following example, a persistent key-value store named “Counts” with key type String and value type Long is created.

    -
    Local State Store
    +
    +StateStoreSupplier countStore = Stores.create("Counts")
    +    .withKeys(Serdes.String())
    +    .withValues(Serdes.Long())
    +    .persistent()
    +    .build();
    +

    - Note that the Processor API is not limited to only accessing the current records as they arrive, but can also maintain local state stores - that keep recently arrived records to use in stateful processing operations such as aggregation or windowed joins. - To take advantage of this local states, developers can use the TopologyBuilder.addStateStore method when building the + To take advantage of these state stores, developers can use the TopologyBuilder.addStateStore method when building the processor topology to create the local state and associate it with the processor nodes that needs to access it; or they can connect a created - local state store with the existing processor nodes through TopologyBuilder.connectProcessorAndStateStores. - -

    -            TopologyBuilder builder = new TopologyBuilder();
    +        state store with the existing processor nodes through TopologyBuilder.connectProcessorAndStateStores.
    +        

    - builder.addSource("SOURCE", "src-topic") +
    +TopologyBuilder builder = new TopologyBuilder();
     
    -                .addProcessor("PROCESS1", MyProcessor1::new, "SOURCE")
    -                // create the in-memory state store "COUNTS" associated with processor "PROCESS1"
    -                .addStateStore(Stores.create("COUNTS").withStringKeys().withStringValues().inMemory().build(), "PROCESS1")
    -                .addProcessor("PROCESS2", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
    -                .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
    +builder.addSource("SOURCE", "src-topic")
     
    -                // connect the state store "COUNTS" with processor "PROCESS2"
    -                .connectProcessorAndStateStores("PROCESS2", "COUNTS");
    +    .addProcessor("PROCESS1", MyProcessor1::new, "SOURCE")
    +    // add the created state store "COUNTS" associated with processor "PROCESS1"
    +    .addStateStore(countStore, "PROCESS1")
    +    .addProcessor("PROCESS2", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
    +    .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
     
    -                .addSink("SINK1", "sink-topic1", "PROCESS1")
    -                .addSink("SINK2", "sink-topic2", "PROCESS2")
    -                .addSink("SINK3", "sink-topic3", "PROCESS3");
    -        
    + // connect the state store "COUNTS" with processor "PROCESS2" + .connectProcessorAndStateStores("PROCESS2", "COUNTS"); -

    + .addSink("SINK1", "sink-topic1", "PROCESS1") + .addSink("SINK2", "sink-topic2", "PROCESS2") + .addSink("SINK3", "sink-topic3", "PROCESS3"); +
    In the next section we present another way to build the processor topology: the Kafka Streams DSL. +
    -

    High-Level Streams DSL

    +

    High-Level Streams DSL

    To build a processor topology using the Streams DSL, developers can apply the KStreamBuilder class, which is extended from the TopologyBuilder. A simple example is included with the source code for Kafka in the streams/examples package. The rest of this section will walk through some code to demonstrate the key steps in creating a topology using the Streams DSL, but we recommend developers to read the full example source - codes for details. + codes for details. -
    KStream and KTable
    - The DSL uses two main abstractions. A KStream is an abstraction of a record stream, where each data record represents a self-contained datum in the unbounded data set. A KTable is an abstraction of a changelog stream, where each data record represents an update. More precisely, the value in a data record is considered to be an update of the last value for the same record key, if any (if a corresponding key doesn't exist yet, the update will be considered a create). To illustrate the difference between KStreams and KTables, let’s imagine the following two data records are being sent to the stream: ("alice", 1) --> ("alice", 3). If these records a KStream and the stream processing application were to sum the values it would return 4. If these records were a KTable, the return would be 3, since the last record would be considered as an update. +

    Duality of Streams and Tables

    +

    + Before we discuss concepts such as aggregations in Kafka Streams we must first introduce tables, and most importantly the relationship between tables and streams: + the so-called stream-table duality. + Essentially, this duality means that a stream can be viewed as a table, and vice versa. Kafka's log compaction feature, for example, exploits this duality. +

    +

    + A simple form of a table is a collection of key-value pairs, also called a map or associative array. Such a table may look as follows: +

    + -
    Create Source Streams from Kafka
    + The stream-table duality describes the close relationship between streams and tables. +
      +
    • Stream as Table: A stream can be considered a changelog of a table, where each data record in the stream captures a state change of the table. A stream is thus a table in disguise, and it can be easily turned into a “real” table by replaying the changelog from beginning to end to reconstruct the table. Similarly, in a more general analogy, aggregating data records in a stream – such as computing the total number of pageviews by user from a stream of pageview events – will return a table (here with the key and the value being the user and its corresponding pageview count, respectively).
    • +
    • Table as Stream: A table can be considered a snapshot, at a point in time, of the latest value for each key in a stream (a stream's data records are key-value pairs). A table is thus a stream in disguise, and it can be easily turned into a “real” stream by iterating over each key-value entry in the table.
    • +
    + +

    + Let's illustrate this with an example. Imagine a table that tracks the total number of pageviews by user (first column of diagram below). Over time, whenever a new pageview event is processed, the state of the table is updated accordingly. Here, the state changes between different points in time – and different revisions of the table – can be represented as a changelog stream (second column). +

    + + +

    + Interestingly, because of the stream-table duality, the same stream can be used to reconstruct the original table (third column): +

    +

    - Either a record stream (defined as KStream) or a changelog stream (defined as KTable) - can be created as a source stream from one or more Kafka topics (for KTable you can only create the source stream + The same mechanism is used, for example, to replicate databases via change data capture (CDC) and, within Kafka Streams, to replicate its so-called state stores across machines for fault-tolerance. + The stream-table duality is such an important concept that Kafka Streams models it explicitly via the KStream, KTable, and GlobalKTable interfaces, which we describe in the next sections. +

    + +
    KStream, KTable, and GlobalKTable
    + The DSL uses three main abstractions. A KStream is an abstraction of a record stream, where each data record represents a self-contained datum in the unbounded data set. + A KTable is an abstraction of a changelog stream, where each data record represents an update. More precisely, the value in a data record is considered to be an update of the last value for the same record key, + if any (if a corresponding key doesn't exist yet, the update will be considered a create). + Like a KTable, a GlobalKTable is an abstraction of a changelog stream, where each data record represents an update. + However, a GlobalKTable is different from a KTable in that it is fully replicated on each KafkaStreams instance. + GlobalKTable also provides the ability to look up current values of data records by keys. + This table-lookup functionality is available through join operations. + + To illustrate the difference between KStreams and KTables/GlobalKTables, let’s imagine the following two data records are being sent to the stream: + +
    +("alice", 1) --> ("alice", 3)
    +
    + + If these records a KStream and the stream processing application were to sum the values it would return 4. If these records were a KTable or GlobalKTable, the return would be 3, since the last record would be considered as an update. + +

    Create Source Streams from Kafka

    + +

    + Either a record stream (defined as KStream) or a changelog stream (defined as KTable or GlobalKTable) + can be created as a source stream from one or more Kafka topics (for KTable and GlobalKTable you can only create the source stream from a single topic).

    -
    -            KStreamBuilder builder = new KStreamBuilder();
    +
    +KStreamBuilder builder = new KStreamBuilder();
     
    -            KStream source1 = builder.stream("topic1", "topic2");
    -            KTable source2 = builder.table("topic3", "stateStoreName");
    -        
    +KStream<String, GenericRecord> source1 = builder.stream("topic1", "topic2"); +KTable<String, GenericRecord> source2 = builder.table("topic3", "stateStoreName"); +GlobalKTable<String, GenericRecord> source2 = builder.globalTable("topic4", "globalStoreName"); +
    -
    Windowing a stream
    +

    Windowing a stream

    A stream processor may need to divide data records into time buckets, i.e. to window the stream by time. This is usually needed for join and aggregation operations, etc. Kafka Streams currently defines the following types of windows:
    • Hopping time windows are windows based on time intervals. They model fixed-sized, (possibly) overlapping windows. A hopping window is defined by two properties: the window's size and its advance interval (aka "hop"). The advance interval specifies by how much a window moves forward relative to the previous one. For example, you can configure a hopping window with a size 5 minutes and an advance interval of 1 minute. Since hopping windows can overlap a data record may belong to more than one such windows.
    • Tumbling time windows are a special case of hopping time windows and, like the latter, are windows based on time intervals. They model fixed-size, non-overlapping, gap-less windows. A tumbling window is defined by a single property: the window's size. A tumbling window is a hopping window whose window size is equal to its advance interval. Since tumbling windows never overlap, a data record will belong to one and only one window.
    • Sliding windows model a fixed-size window that slides continuously over the time axis; here, two data records are said to be included in the same window if the difference of their timestamps is within the window size. Thus, sliding windows are not aligned to the epoch, but on the data record timestamps. In Kafka Streams, sliding windows are used only for join operations, and can be specified through the JoinWindows class.
    • +
    • Session windows are used to aggregate key-based events into sessions. + Sessions represent a period of activity separated by a defined gap of inactivity. + Any events processed that fall within the inactivity gap of any existing sessions are merged into the existing sessions. + If the event falls outside of the session gap, then a new session will be created. + Session windows are tracked independently across keys (e.g. windows of different keys typically have different start and end times) and their sizes vary (even windows for the same key typically have different sizes); + as such session windows can't be pre-computed and are instead derived from analyzing the timestamps of the data records. +
    -
    Joins
    +

    + In the Kafka Streams DSL users can specify a retention period for the window. This allows Kafka Streams to retain old window buckets for a period of time in order to wait for the late arrival of records whose timestamps fall within the window interval. + If a record arrives after the retention period has passed, the record cannot be processed and is dropped. +

    + +

    + Late-arriving records are always possible in real-time data streams. However, it depends on the effective time semantics how late records are handled. Using processing-time, the semantics are “when the data 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 only really can 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. +

    + +

    Join multiple streams

    A join operation merges two streams based on the keys of their data records, and yields a new stream. A join over record streams usually needs to be performed on a windowing basis because otherwise the number of records that must be maintained for performing the join may grow indefinitely. In Kafka Streams, you may perform the following join operations:
    • KStream-to-KStream Joins are always windowed joins, since otherwise the memory and state required to compute the join would grow infinitely in size. Here, a newly received record from one of the streams is joined with the other stream's records within the specified window interval to produce one result for each matching pair based on user-provided ValueJoiner. A new KStream instance representing the result stream of the join is returned from this operator.
    • KTable-to-KTable Joins are join operations designed to be consistent with the ones in relational databases. Here, both changelog streams are materialized into local state stores first. When a new record is received from one of the streams, it is joined with the other stream's materialized state stores to produce one result for each matching pair based on user-provided ValueJoiner. A new KTable instance representing the result stream of the join, which is also a changelog stream of the represented table, is returned from this operator.
    • -
    • KStream-to-KTable Joins allow you to perform table lookups against a changelog stream (KTable) upon receiving a new record from another record stream (KStream). An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (KTable). Only records received from the record stream will trigger the join and produce results via ValueJoiner, not vice versa (i.e., records received from the changelog stream will be used only to update the materialized state store). A new KStream instance representing the result stream of the join is returned from this operator.
    • +
    • KStream-to-KTable Joins allow you to perform table lookups against a changelog stream (KTable) upon receiving a new record from another record stream (KStream). An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (KTable). Only records received from the record stream will trigger the join and produce results via ValueJoiner, not vice versa (i.e., records received from the changelog stream will be used only to update the materialized state store). A new KStream instance representing the result stream of the join is returned from this operator.
    • +
    • KStream-to-GlobalKTable Joins allow you to perform table lookups against a fully replicated changelog stream (GlobalKTable) upon receiving a new record from another record stream (KStream). + Joins with a GlobalKTable don't require repartitioning of the input KStream as all partitions of the GlobalKTable are available on every KafkaStreams instance. + The KeyValueMapper provided with the join operation is applied to each KStream record to extract the join-key that is used to do the lookup to the GlobalKTable so non-record-key joins are possible. + An example use case would be to enrich a stream of user activities (KStream) with the latest user profile information (GlobalKTable). + Only records received from the record stream will trigger the join and produce results via ValueJoiner, not vice versa (i.e., records received from the changelog stream will be used only to update the materialized state store). + A new KStream instance representing the result stream of the join is returned from this operator.
    - Depending on the operands the following join operations are supported: inner joins, outer joins and left joins. Their semantics are similar to the corresponding operators in relational databases. - a -
    Transform a stream
    + Depending on the operands the following join operations are supported: inner joins, outer joins and left joins. + Their semantics are similar to the corresponding operators in relational databases. + +
    Aggregate a stream
    + An aggregation operation takes one input stream, and yields a new stream by combining multiple input records into a single output record. Examples of aggregations are computing counts or sum. An aggregation over record streams usually needs to be performed on a windowing basis because otherwise the number of records that must be maintained for performing the aggregation may grow indefinitely.

    - There is a list of transformation operations provided for KStream and KTable respectively. + 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 simply 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. +

    + +

    Transform a stream

    + +

    + Besides join and aggregation operations, there is a list of other transformation operations provided for KStream and KTable respectively. Each of these operations may generate either one or more KStream and KTable objects and can be translated into one or more connected processors into the underlying processor topology. All these transformation methods can be chained together to compose a complex processor topology. @@ -340,10 +606,10 @@

    Transform a stre

    -
    -            // written in Java 8+, using lambda expressions
    -            KStream mapped = source1.mapValue(record -> record.get("category"));
    -        
    +
    +// written in Java 8+, using lambda expressions
    +KStream<String, GenericRecord> mapped = source1.mapValue(record -> record.get("category"));
    +

    Stateless transformations, by definition, do not depend on any state for processing, and hence implementation-wise @@ -354,50 +620,312 @@

    Transform a stre based on them.

    -
    -            // written in Java 8+, using lambda expressions
    -            KTable, Long> counts = source1.groupByKey().aggregate(
    -                () -> 0L,  // initial value
    -                (aggKey, value, aggregate) -> aggregate + 1L,   // aggregating value
    -                TimeWindows.of("counts", 5000L).advanceBy(1000L), // intervals in milliseconds
    -                Serdes.Long() // serde for aggregated value
    -            );
    +
    +// written in Java 8+, using lambda expressions
    +KTable<Windowed<String>, Long> counts = source1.groupByKey().aggregate(
    +    () -> 0L,  // initial value
    +    (aggKey, value, aggregate) -> aggregate + 1L,   // aggregating value
    +    TimeWindows.of("counts", 5000L).advanceBy(1000L), // intervals in milliseconds
    +    Serdes.Long() // serde for aggregated value
    +);
     
    -            KStream joined = source1.leftJoin(source2,
    -                (record1, record2) -> record1.get("user") + "-" + record2.get("region");
    -            );
    -        
    +KStream<String, String> joined = source1.leftJoin(source2, + (record1, record2) -> record1.get("user") + "-" + record2.get("region"); +); +
    -
    Write streams back to Kafka
    +

    Write streams back to Kafka

    At the end of the processing, users can choose to (continuously) write the final resulted streams back to a Kafka topic through KStream.to and KTable.to.

    -
    -            joined.to("topic4");
    -        
    +
    +joined.to("topic4");
    +
    If your application needs to continue reading and processing the records after they have been materialized to a topic via to above, one option is to construct a new stream that reads from the output topic; Kafka Streams provides a convenience method called through: -
    -            // equivalent to
    -            //
    -            // joined.to("topic4");
    -            // materialized = builder.stream("topic4");
    -            KStream materialized = joined.through("topic4");
    -        
    +
    +// equivalent to
    +//
    +// joined.to("topic4");
    +// materialized = builder.stream("topic4");
    +KStream<String, String> materialized = joined.through("topic4");
    +
    +
    +

    Application Configuration and Execution

    -

    Besides defining the topology, developers will also need to configure their applications in StreamsConfig before running it. A complete list of - Kafka Streams configs can be found here. + Kafka Streams configs can be found here. +

    + +

    + Specifying the configuration in Kafka Streams is similar to the Kafka Producer and Consumer clients. Typically, you create a java.util.Properties instance, + set the necessary parameters, and construct a StreamsConfig instance from the Properties instance.

    + +
    +import java.util.Properties;
    +import org.apache.kafka.streams.StreamsConfig;
    +
    +Properties settings = new Properties();
    +// Set a few key parameters
    +settings.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-first-streams-application");
    +settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker1:9092");
    +settings.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "zookeeper1:2181");
    +// Any further settings
    +settings.put(... , ...);
    +
    +// Create an instance of StreamsConfig from the Properties instance
    +StreamsConfig config = new StreamsConfig(settings);
    +
    + +

    + Apart from Kafka Streams' own configuration parameters you can also specify parameters for the Kafka consumers and producers that are used internally, + depending on the needs of your application. Similar to the Streams settings you define any such consumer and/or producer settings via StreamsConfig. + Note that some consumer and producer configuration parameters do use the same parameter name. For example, send.buffer.bytes or receive.buffer.bytes which + are used to configure TCP buffers; request.timeout.ms and retry.backoff.ms which control retries for client request (and some more). + If you want to set different values for consumer and producer for such a parameter, you can prefix the parameter name with consumer. or producer.: +

    + +
    +Properties settings = new Properties();
    +// Example of a "normal" setting for Kafka Streams
    +settings.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "kafka-broker-01:9092");
    +
    +// Customize the Kafka consumer settings
    +streamsSettings.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, 60000);
    +
    +// Customize a common client setting for both consumer and producer
    +settings.put(CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG, 100L);
    +
    +// Customize different values for consumer and producer
    +settings.put("consumer." + ConsumerConfig.RECEIVE_BUFFER_CONFIG, 1024 * 1024);
    +settings.put("producer." + ProducerConfig.RECEIVE_BUFFER_CONFIG, 64 * 1024);
    +// Alternatively, you can use
    +settings.put(StreamsConfig.consumerPrefix(ConsumerConfig.RECEIVE_BUFFER_CONFIG), 1024 * 1024);
    +settings.put(StremasConfig.producerConfig(ProducerConfig.RECEIVE_BUFFER_CONFIG), 64 * 1024);
    +
    + +

    + You can call Kafka Streams from anywhere in your application code. + Very commonly though you would do so within the main() method of your application, or some variant thereof. +

    + +

    + First, you must create an instance of KafkaStreams. The first argument of the KafkaStreams constructor takes a topology + builder (either KStreamBuilder for the Kafka Streams DSL, or TopologyBuilder for the Processor API) + that is used to define a topology; The second argument is an instance of StreamsConfig mentioned above. +

    + +
    +import org.apache.kafka.streams.KafkaStreams;
    +import org.apache.kafka.streams.StreamsConfig;
    +import org.apache.kafka.streams.kstream.KStreamBuilder;
    +import org.apache.kafka.streams.processor.TopologyBuilder;
    +
    +// Use the builders to define the actual processing topology, e.g. to specify
    +// from which input topics to read, which stream operations (filter, map, etc.)
    +// should be called, and so on.
    +
    +KStreamBuilder builder = ...;  // when using the Kafka Streams DSL
    +//
    +// OR
    +//
    +TopologyBuilder builder = ...; // when using the Processor API
    +
    +// Use the configuration to tell your application where the Kafka cluster is,
    +// which serializers/deserializers to use by default, to specify security settings,
    +// and so on.
    +StreamsConfig config = ...;
    +
    +KafkaStreams streams = new KafkaStreams(builder, config);
    +
    + +

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

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

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

    + +
    +streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
    +    public uncaughtException(Thread t, throwable e) {
    +        // here you should examine the exception and perform an appropriate action!
    +    }
    +);
    +
    + +

    + To stop the application instance call the close() method: +

    + +
    +// Stop the Kafka Streams instance
    +streams.close();
    +
    + + Now it's time to execute your application that uses the Kafka Streams library, which can be run just like any other Java application – there is no special magic or requirement on the side of Kafka Streams. + For example, you can package your Java application as a fat jar file and then start the application via: + +
    +# Start the application in class `com.example.MyStreamsApp`
    +# from the fat jar named `path-to-app-fatjar.jar`.
    +$ java -cp path-to-app-fatjar.jar com.example.MyStreamsApp
    +
    + +

    + When the application instance starts running, the defined processor topology will be initialized as one or more stream tasks that can be executed in parallel by the stream threads within the instance. + If the processor topology defines any state stores, these state stores will also be (re-)constructed, if possible, during the initialization + period of their associated stream tasks. + It is important to understand that, when starting your application as described above, you are actually launching what Kafka Streams considers to be one instance of your application. + More than one instance of your application may be running at a time, and in fact the common scenario is that there are indeed multiple instances of your application running in parallel (e.g., on another JVM or another machine). + In such cases, Kafka Streams transparently re-assigns tasks from the existing instances to the new instance that you just started. + See Stream Partitions and Tasks and Threading Model for details. +

    +
    + +

    Upgrade Guide and API Changes

    + +

    + If you want to upgrade from 0.10.1.x to 0.10.2, see the Upgrade Section for 0.10.2. + It highlights incompatible changes you need to consider to upgrade your code and application. + See below a complete list of 0.10.2 API and semantical changes that allow you to advance your application and/or simplify your code base, including the usage of new features. +

    + +

    + If you want to upgrade from 0.10.0.x to 0.10.1, see the Upgrade Section for 0.10.1. + It highlights incompatible changes you need to consider to upgrade your code and application. + See below a complete list of 0.10.1 API changes that allow you to advance your application and/or simplify your code base, including the usage of new features. +

    + +

    Notable changes in 0.10.2.1

    +

    + Parameter updates in StreamsConfig: +

    +
      +
    • of particular importance to improve the resiliency of a Kafka Streams application are two changes to default parameters of producer retries and consumer max.poll.interval.ms
    • +
    +

    Streams API changes in 0.10.2.0

    + +

    + New methods in KafkaStreams: +

    +
      +
    • set a listener to react on application state change via #setStateListener(StateListener listener)
    • +
    • retrieve the current application state via #state()
    • +
    • retrieve the global metrics registry via #metrics()
    • +
    • apply a timeout when closing an application via #close(long timeout, TimeUnit timeUnit)
    • +
    • specify a custom indent when retrieving Kafka Streams information via #toString(String indent)
    • +
    + +

    + Parameter updates in StreamsConfig: +

    +
      +
    • parameter zookeeper.connect was deprecated; a Kafka Streams application does no longer interact with Zookeeper for topic management but uses the new broker admin protocol + (cf. KIP-4, Section "Topic Admin Schema")
    • +
    • added many new parameters for metrics, security, and client configurations
    • +
    + +

    Changes in StreamsMetrics interface:

    +
      +
    • removed methods: #addLatencySensor()
    • +
    • added methods: #addLatencyAndThroughputSensor(), #addThroughputSensor(), #recordThroughput(), + #addSensor(), #removeSensor()
    • +
    + +

    New methods in TopologyBuilder:

    +
      +
    • added overloads for #addSource() that allow to define a auto.offset.reset policy per source node
    • +
    • added methods #addGlobalStore() to add global StateStores
    • +
    + +

    New methods in KStreamBuilder:

    +
      +
    • added overloads for #stream() and #table() that allow to define a auto.offset.reset policy per input stream/table
    • +
    • added method #globalKTable() to create a GlobalKTable
    • +
    + +

    New joins for KStream:

    +
      +
    • added overloads for #join() to join with KTable
    • +
    • added overloads for #join() and leftJoin() to join with GlobalKTable
    • +
    • note, join semantics in 0.10.2 were improved and thus you might see different result compared to 0.10.0.x and 0.10.1.x + (cf. Kafka Streams Join Semantics in the Apache Kafka wiki) +
    + +

    Aligned null-key handling for KTable joins:

    +
      +
    • like all other KTable operations, KTable-KTable joins do not throw an exception on null key records anymore, but drop those records silently
    • +
    + +

    New window type Session Windows:

    +
      +
    • added class SessionWindows to specify session windows
    • +
    • added overloads for KGroupedStream methods #count(), #reduce(), and #aggregate() + to allow session window aggregations
    • +
    + +

    Changes to TimestampExtractor:

    +
      +
    • method #extract() has a second parameter now
    • +
    • new default timestamp extractor class FailOnInvalidTimestamp + (it gives the same behavior as old (and removed) default extractor ConsumerRecordTimestampExtractor)
    • +
    • new alternative timestamp extractor classes LogAndSkipOnInvalidTimestamp and UsePreviousTimeOnInvalidTimestamps
    • +
    + +

    Relaxed type constraints of many DSL interfaces, classes, and methods (cf. KIP-100).

    + +

    Streams API changes in 0.10.1.0

    + +

    Stream grouping and aggregation split into two methods:

    +
      +
    • old: KStream #aggregateByKey(), #reduceByKey(), and #countByKey()
    • +
    • new: KStream#groupByKey() plus KGroupedStream #aggregate(), #reduce(), and #count()
    • +
    • Example: stream.countByKey() changes to stream.groupByKey().count()
    • +
    + +

    Auto Repartitioning:

    +
      +
    • a call to through() after a key-changing operator and before an aggregation/join is no longer required
    • +
    • Example: stream.selectKey(...).through(...).countByKey() changes to stream.selectKey().groupByKey().count()
    • +
    + +

    TopologyBuilder:

    +
      +
    • methods #sourceTopics(String applicationId) and #topicGroups(String applicationId) got simplified to #sourceTopics() and #topicGroups()
    • +
    + +

    DSL: new parameter to specify state store names:

    +
      +
    • The new Interactive Queries feature requires to specify a store name for all source KTables and window aggregation result KTables (previous parameter "operator/window name" is now the storeName)
    • +
    • KStreamBuilder#table(String topic) changes to #topic(String topic, String storeName)
    • +
    • KTable#through(String topic) changes to #through(String topic, String storeName)
    • +
    • KGroupedStream #aggregate(), #reduce(), and #count() require additional parameter "String storeName"
    • +
    • Example: stream.countByKey(TimeWindows.of("windowName", 1000)) changes to stream.groupByKey().count(TimeWindows.of(1000), "countStoreName")
    • +
    + +

    Windowing:

    +
      +
    • Windows are not named anymore: TimeWindows.of("name", 1000) changes to TimeWindows.of(1000) (cf. DSL: new parameter to specify state store names)
    • +
    • JoinWindows has no default size anymore: JoinWindows.of("name").within(1000) changes to JoinWindows.of(1000)
    • +
    + diff --git a/docs/toc.html b/docs/toc.html new file mode 100644 index 0000000000000..935703bc9072a --- /dev/null +++ b/docs/toc.html @@ -0,0 +1,160 @@ + + + + + + +
    diff --git a/docs/upgrade.html b/docs/upgrade.html index a6683ba8b16cc..0221f63829463 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -14,29 +14,89 @@ See the License for the specific language governing permissions and limitations under the License. --> + + + + + +
    diff --git a/docs/uses.html b/docs/uses.html index b86d917a265fb..4e88859229c79 100644 --- a/docs/uses.html +++ b/docs/uses.html @@ -15,40 +15,67 @@ limitations under the License. --> -

    Here is a description of a few of the popular use cases for Apache Kafka. For an overview of a number of these areas in action, see this blog post.

    +

    Here is a description of a few of the popular use cases for Apache Kafka™. +For an overview of a number of these areas in action, see this blog post.

    Messaging

    -Kafka works well as a replacement for a more traditional message broker. Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good solution for large scale message processing applications. +Kafka works well as a replacement for a more traditional message broker. +Message brokers are used for a variety of reasons (to decouple processing from data producers, to buffer unprocessed messages, etc). +In comparison to most messaging systems Kafka has better throughput, built-in partitioning, replication, and fault-tolerance which makes it a good +solution for large scale message processing applications.

    -In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong durability guarantees Kafka provides. +In our experience messaging uses are often comparatively low-throughput, but may require low end-to-end latency and often depend on the strong +durability guarantees Kafka provides.

    -In this domain Kafka is comparable to traditional messaging systems such as ActiveMQ or RabbitMQ. +In this domain Kafka is comparable to traditional messaging systems such as ActiveMQ or +RabbitMQ.

    Website Activity Tracking

    -The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or offline data warehousing systems for offline processing and reporting. +The original use case for Kafka was to be able to rebuild a user activity tracking pipeline as a set of real-time publish-subscribe feeds. +This means site activity (page views, searches, or other actions users may take) is published to central topics with one topic per activity type. +These feeds are available for subscription for a range of use cases including real-time processing, real-time monitoring, and loading into Hadoop or +offline data warehousing systems for offline processing and reporting.

    Activity tracking is often very high volume as many activity messages are generated for each user page view.

    Metrics

    -Kafka is often used for operational monitoring data. This involves aggregating statistics from distributed applications to produce centralized feeds of operational data. +Kafka is often used for operational monitoring data. +This involves aggregating statistics from distributed applications to produce centralized feeds of operational data.

    Log Aggregation

    -Many people use Kafka as a replacement for a log aggregation solution. Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption. +Many people use Kafka as a replacement for a log aggregation solution. +Log aggregation typically collects physical log files off servers and puts them in a central place (a file server or HDFS perhaps) for processing. +Kafka abstracts away the details of files and gives a cleaner abstraction of log or event data as a stream of messages. +This allows for lower-latency processing and easier support for multiple data sources and distributed data consumption. -In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, and much lower end-to-end latency. +In comparison to log-centric systems like Scribe or Flume, Kafka offers equally good performance, stronger durability guarantees due to replication, +and much lower end-to-end latency.

    Stream Processing

    -Many users of Kafka process data in processing pipelines consisting of multiple stages, where raw input data is consumed from Kafka topics and then aggregated, enriched, or otherwise transformed into new topics for further consumption or follow-up processing. For example, a processing pipeline for recommending news articles might crawl article content from RSS feeds and publish it to an "articles" topic; further processing might normalize or deduplicate this content and published the cleansed article content to a new topic; a final processing stage might attempt to recommend this content to users. Such processing pipelines create graphs of real-time data flows based on the individual topics. Starting in 0.10.0.0, a light-weight but powerful stream processing library called Kafka Streams is available in Apache Kafka to perform such data processing as described above. Apart from Kafka Streams, alternative open source stream processing tools include Apache Storm and Apache Samza. +Many users of Kafka process data in processing pipelines consisting of multiple stages, where raw input data is consumed from Kafka topics and then +aggregated, enriched, or otherwise transformed into new topics for further consumption or follow-up processing. +For example, a processing pipeline for recommending news articles might crawl article content from RSS feeds and publish it to an "articles" topic; +further processing might normalize or deduplicate this content and published the cleansed article content to a new topic; +a final processing stage might attempt to recommend this content to users. +Such processing pipelines create graphs of real-time data flows based on the individual topics. +Starting in 0.10.0.0, a light-weight but powerful stream processing library called Kafka Streams +is available in Apache Kafka to perform such data processing as described above. +Apart from Kafka Streams, alternative open source stream processing tools include Apache Storm and +Apache Samza.

    Event Sourcing

    -Event sourcing is a style of application design where state changes are logged as a time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style. +Event sourcing is a style of application design where state changes are logged as a +time-ordered sequence of records. Kafka's support for very large stored log data makes it an excellent backend for an application built in this style.

    Commit Log

    -Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing mechanism for failed nodes to restore their data. The log compaction feature in Kafka helps support this usage. In this usage Kafka is similar to Apache BookKeeper project. +Kafka can serve as a kind of external commit-log for a distributed system. The log helps replicate data between nodes and acts as a re-syncing +mechanism for failed nodes to restore their data. +The log compaction feature in Kafka helps support this usage. +In this usage Kafka is similar to Apache BookKeeper project. diff --git a/gradle.properties b/gradle.properties index 86aedc35ca961..02f1935d7b379 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ group=org.apache.kafka # NOTE: When you change this version number, you should also make sure to update # the version numbers in tests/kafkatest/__init__.py and kafka-merge-pr.py. -version=0.10.2.0-SNAPSHOT +version=0.10.2.2-SNAPSHOT scalaVersion=2.10.6 task=build org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m diff --git a/jenkins.sh b/jenkins.sh new file mode 100755 index 0000000000000..a21da0546874d --- /dev/null +++ b/jenkins.sh @@ -0,0 +1,20 @@ +#!/bin/bash +# 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. + +# This script is used for verifying changes in Jenkins. In order to provide faster feedback, the tasks are ordered so +# that faster tasks are executed in every module before slower tasks (if possible). For example, the unit tests for all +# the modules are executed before the integration tests. +./gradlew clean compileJava compileScala compileTestJava compileTestScala checkstyleMain checkstyleTest test --no-daemon -Dorg.gradle.project.testLoggingEvents=started,passed,skipped,failed "$@" diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 47567223862e8..7a930756182d9 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -42,12 +42,13 @@ * using specific data types (here: JSON POJO; but can also be Avro specific bindings, etc.) for serdes * in Kafka Streams. * - * In this example, we join a stream of pageviews (aka clickstreams) that reads from a topic named "streams-pageview-input" + * In this example, we join a stream of pageviews (aka clickstreams) that reads from a topic named "streams-pageview-input" * with a user profile table that reads from a topic named "streams-userprofile-input", where the data format * is JSON string representing a record in the stream or table, to compute the number of pageviews per user region. * - * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) - * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. + * Before running this example you must create the input topics and the output topic (e.g. via + * bin/kafka-topics.sh --create ...), and write some data to the input topics (e.g. via + * bin/kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ public class PageViewTypedDemo { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 67e8cc50c7dfa..68becdef0e63c 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -50,8 +50,9 @@ * with a user profile table that reads from a topic named "streams-userprofile-input", where the data format * is JSON string representing a record in the stream or table, to compute the number of pageviews per user region. * - * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) - * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. + * Before running this example you must create the input topics and the output topic (e.g. via + * bin/kafka-topics.sh --create ...), and write some data to the input topics (e.g. via + * bin/kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ public class PageViewUntypedDemo { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index 3c1bd8c049a67..6626f4597d2e4 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -32,8 +32,9 @@ * In this example, we implement a simple "pipe" program that reads from a source topic "streams-file-input" * and writes the data as-is (i.e. unmodified) into a sink topic "streams-pipe-output". * - * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) - * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. + * Before running this example you must create the input topic and the output topic (e.g. via + * bin/kafka-topics.sh --create ...), and write some data to the input topic (e.g. via + * bin/kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ public class PipeDemo { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 99fbc1574b55d..892c66cb3416f 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -40,8 +40,9 @@ * represent lines of text; and the histogram output is written to topic "streams-wordcount-output" where each record * is an updated count of a single word. * - * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) - * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. + * Before running this example you must create the input topic and the output topic (e.g. via + * bin/kafka-topics.sh --create ...), and write some data to the input topic (e.g. via + * bin/kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ public class WordCountDemo { @@ -53,6 +54,8 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data + // Note: To re-run the demo, you need to use the offset reset tool: + // https://cwiki.apache.org/confluence/display/KAFKA/Kafka+Streams+Application+Reset+Tool props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index e0560a9c6fd49..aefaf41e6f468 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -41,8 +41,9 @@ * represent lines of text; and the histogram output is written to topic "streams-wordcount-processor-output" where each record * is an updated count of a single word. * - * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) - * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. + * Before running this example you must create the input topic and the output topic (e.g. via + * bin/kafka-topics.sh --create ...), and write some data to the input topic (e.g. via + * bin/kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ public class WordCountProcessorDemo { @@ -95,9 +96,7 @@ public void punctuate(long timestamp) { } @Override - public void close() { - this.kvStore.close(); - } + public void close() {} }; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java index e0312f92304aa..ed3d4884a4b57 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaClientSupplier.java @@ -17,31 +17,43 @@ package org.apache.kafka.streams; -import java.util.Map; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.StreamThread; + +import java.util.Map; +/** + * {@code KafkaClientSupplier} can be used to provide custom Kafka clients to a {@link KafkaStreams} instance. + * + * @see KafkaStreams#KafkaStreams(org.apache.kafka.streams.processor.TopologyBuilder, StreamsConfig, KafkaClientSupplier) + */ public interface KafkaClientSupplier { /** - * Creates an instance of Producer which is used to produce records. - * @param config producer config which supplied by {@link StreamsConfig} given to {@link KafkaStreams} - * @return an instance of kafka Producer + * Create a {@link Producer} which is used to write records to sink topics. + * + * @param config {@link StreamsConfig#getProducerConfigs(String) producer config} which is supplied by the + * {@link StreamsConfig} given to the {@link KafkaStreams} instance + * @return an instance of Kafka producer */ - Producer getProducer(Map config); + Producer getProducer(final Map config); /** - * Creates an instance of Consumer which is used to consume records of source topics. - * @param config consumer config which supplied by {@link StreamsConfig} given to {@link KafkaStreams} - * @return an instance of kafka Consumer + * Create a {@link Consumer} which is used to read records of source topics. + * + * @param config {@link StreamsConfig#getConsumerConfigs(StreamThread, String, String) consumer config} which is + * supplied by the {@link StreamsConfig} given to the {@link KafkaStreams} instance + * @return an instance of Kafka consumer */ - Consumer getConsumer(Map config); + Consumer getConsumer(final Map config); /** - * Creates an instance of Consumer which is used to consume records of internal topics. - * @param config restore consumer config which supplied by {@link StreamsConfig} given to - * {@link KafkaStreams} - * @return an instance of kafka Consumer + * Create a {@link Consumer} which is used to read records to restore {@link StateStore}s. + * + * @param config {@link StreamsConfig#getRestoreConsumerConfigs(String) restore consumer config} which is supplied + * by the {@link StreamsConfig} given to the {@link KafkaStreams} + * @return an instance of Kafka consumer */ - Consumer getRestoreConsumer(Map config); + Consumer getRestoreConsumer(final Map config); } diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 31456b2a16086..cd0e86153103c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -17,6 +17,9 @@ package org.apache.kafka.streams; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.annotation.InterfaceStability; @@ -28,6 +31,12 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.TopologyBuilder; @@ -36,6 +45,7 @@ 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.StreamsKafkaClient; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.QueryableStoreType; @@ -48,60 +58,62 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.IOException; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Properties; -import java.util.UUID; import java.util.Set; -import java.util.HashSet; -import java.util.Arrays; +import java.util.UUID; import java.util.concurrent.TimeUnit; import static org.apache.kafka.common.utils.Utils.getHost; import static org.apache.kafka.common.utils.Utils.getPort; /** - * Kafka Streams allows for performing continuous computation on input coming from one or more input topics and - * sends output to zero or more output topics. + * A Kafka client that allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero, one, or more output topics. *

    - * The computational logic can be specified either by using the {@link TopologyBuilder} class to define the a DAG topology of - * {@link org.apache.kafka.streams.processor.Processor}s or by using the {@link org.apache.kafka.streams.kstream.KStreamBuilder} - * class which provides the high-level {@link org.apache.kafka.streams.kstream.KStream} DSL to define the transformation. + * The computational logic can be specified either by using the {@link TopologyBuilder} to define a DAG topology of + * {@link Processor}s or by using the {@link KStreamBuilder} which provides the high-level DSL to define transformations. *

    - * The {@link KafkaStreams} class manages the lifecycle of a Kafka Streams instance. One stream instance can contain one or - * more threads specified in the configs for the processing work. + * One {@code KafkaStreams} instance can contain one or more threads specified in the configs for the processing work. *

    - * A {@link KafkaStreams} instance can co-ordinate with any other instances with the same application ID (whether in this same process, on other processes - * on this machine, or on remote machines) as a single (possibly distributed) stream processing client. These instances will divide up the work - * based on the assignment of the input topic partitions so that all partitions are being - * consumed. If instances are added or failed, all instances will rebalance the partition assignment among themselves - * to balance processing load. - *

    - * Internally the {@link KafkaStreams} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} - * and {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer} instance that is used for reading input and writing output. + * A {@code KafkaStreams} instance can co-ordinate with any other instances with the same + * {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} (whether in the same process, on other processes on this + * machine, or on remote machines) as a single (possibly distributed) stream processing application. + * These instances will divide up the work based on the assignment of the input topic partitions so that all partitions + * are being consumed. + * If instances are added or fail, all (remaining) instances will rebalance the partition assignment among themselves + * to balance processing load and ensure that all input topic partitions are processed. *

    + * Internally a {@code KafkaStreams} instance contains a normal {@link KafkaProducer} and {@link KafkaConsumer} instance + * that is used for reading input and writing output. *

    * A simple example might look like this: - *

    - *    Map<String, Object> props = new HashMap<>();
    - *    props.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-stream-processing-application");
    - *    props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
    - *    props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
    - *    props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName());
    - *    StreamsConfig config = new StreamsConfig(props);
    + * 
    {@code
    + * Map props = new HashMap<>();
    + * props.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-stream-processing-application");
    + * props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
    + * props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass());
    + * props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass());
    + * StreamsConfig config = new StreamsConfig(props);
    + *
    + * KStreamBuilder builder = new KStreamBuilder();
    + * builder.stream("my-input-topic").mapValues(value -> value.length().toString()).to("my-output-topic");
      *
    - *    KStreamBuilder builder = new KStreamBuilder();
    - *    builder.stream("my-input-topic").mapValues(value -> value.length().toString()).to("my-output-topic");
    + * KafkaStreams streams = new KafkaStreams(builder, config);
    + * streams.start();
    + * }
    * - * KafkaStreams streams = new KafkaStreams(builder, config); - * streams.start(); - *
    + * @see KStreamBuilder + * @see TopologyBuilder */ - @InterfaceStability.Unstable public class KafkaStreams { @@ -120,6 +132,7 @@ public class KafkaStreams { // of the co-location of stream thread's consumers. It is for internal // usage only and should not be exposed to users at all. private final UUID processId; + private final String logPrefix; private final StreamsMetadataState streamsMetadataState; private final StreamsConfig config; @@ -131,111 +144,122 @@ public class KafkaStreams { * Note this instance will be in "Rebalancing" state if any of its threads is rebalancing * The expected state transition with the following defined states is: * - * +-----------+ - * +<------|Created | - * | +-----+-----+ - * | | +--+ - * | v | | - * | +-----+---v--+--+ - * +<----- | Rebalancing |<--------+ - * | +-----+---------+ ^ - * | +--+ | - * | | | | - * | +-----+---v--+-----+ | - * +------>|Running |------+ - * | +-----+------------+ + *
    +     *                 +--------------+
    +     *         +<----- | Created      |
    +     *         |       +-----+--------+
    +     *         |             |
    +     *         |             v
    +     *         |       +-----+--------+
    +     *         +<----- | Rebalancing  | <----+
    +     *         |       +--------------+      |
    +     *         |                             |
    +     *         |                             |
    +     *         |       +--------------+      |
    +     *         +-----> | Running      | ---->+
    +     *         |       +-----+--------+
          *         |             |
          *         |             v
    -     *         |     +-------+--------+
    -     *         +---->|Pending         |
    -     *               |Shutdown        |
    -     *               +-------+--------+
    +     *         |       +-----+--------+
    +     *         +-----> | Pending      |
    +     *                 | Shutdown     |
    +     *                 +-----+--------+
          *                       |
          *                       v
    -     *                 +-----+-----+
    -     *                 |Not Running|
    -     *                 +-----------+
    +     *                 +-----+--------+
    +     *                 | Not Running  |
    +     *                 +--------------+
    +     * 
    */ public enum State { CREATED(1, 2, 3), RUNNING(2, 3), REBALANCING(1, 2, 3), PENDING_SHUTDOWN(4), NOT_RUNNING; private final Set validTransitions = new HashSet<>(); - State(final Integer...validTransitions) { + State(final Integer... validTransitions) { this.validTransitions.addAll(Arrays.asList(validTransitions)); } public boolean isRunning() { - return this.equals(RUNNING) || this.equals(REBALANCING); + return equals(RUNNING) || equals(REBALANCING); } public boolean isCreatedOrRunning() { - return isRunning() || this.equals(CREATED); + return isRunning() || equals(CREATED); } public boolean isValidTransition(final State newState) { return validTransitions.contains(newState.ordinal()); } } - private volatile State state = KafkaStreams.State.CREATED; + private volatile State state = State.CREATED; private StateListener stateListener = null; - private final StreamStateListener streamStateListener = new StreamStateListener(); + /** - * Listen to state change events + * Listen to {@link State} change events. */ public interface StateListener { /** - * Called when state changes - * @param newState current state - * @param oldState previous state + * Called when state changes. + * + * @param newState new state + * @param oldState previous state */ void onChange(final State newState, final State oldState); } /** - * An app can set {@link StateListener} so that the app is notified when state changes - * @param listener + * An app can set a single {@link StateListener} so that the app is notified when state changes. + * @param listener a new state listener */ public void setStateListener(final StateListener listener) { - this.stateListener = listener; + stateListener = listener; } - private synchronized void setState(State newState) { - State oldState = state; + private synchronized void setState(final State newState) { + final State oldState = state; if (!state.isValidTransition(newState)) { - log.warn("Unexpected state transition from " + state + " to " + newState); + log.warn("{} Unexpected state transition from {} to {}.", logPrefix, oldState, newState); + } else { + log.info("{} State transition from {} to {}.", logPrefix, oldState, newState); } + state = newState; + if (stateListener != null) { stateListener.onChange(state, oldState); } } - /** - * @return The state this instance is in + * Return the current {@link State} of this {@code KafkaStreams} instance. + * + * @return the currnt state of this Kafka Streams instance */ public synchronized State state() { return state; } /** - * Get read-only handle on global metrics registry + * Get read-only handle on global metrics registry. + * * @return Map of all metrics. */ public Map metrics() { - return Collections.unmodifiableMap(this.metrics.metrics()); + return Collections.unmodifiableMap(metrics.metrics()); } private class StreamStateListener implements StreamThread.StateListener { @Override - public synchronized void onChange(final StreamThread thread, final StreamThread.State newState, final StreamThread.State oldState) { + public synchronized void onChange(final StreamThread thread, + final StreamThread.State newState, + final StreamThread.State oldState) { threadState.put(thread.getId(), newState); if (newState == StreamThread.State.PARTITIONS_REVOKED || newState == StreamThread.State.ASSIGNING_PARTITIONS) { setState(State.REBALANCING); } else if (newState == StreamThread.State.RUNNING) { - for (StreamThread.State state : threadState.values()) { + for (final StreamThread.State state : threadState.values()) { if (state != StreamThread.State.RUNNING) { return; } @@ -244,35 +268,38 @@ public synchronized void onChange(final StreamThread thread, final StreamThread. } } } + /** - * Construct the stream instance. + * Create a {@code KafkaStreams} instance. * * @param builder the processor topology builder specifying the computational logic - * @param props properties for the {@link StreamsConfig} + * @param props properties for {@link StreamsConfig} */ public KafkaStreams(final TopologyBuilder builder, final Properties props) { this(builder, new StreamsConfig(props), new DefaultKafkaClientSupplier()); } /** - * Construct the stream instance. + * Create a {@code KafkaStreams} instance. * * @param builder the processor topology builder specifying the computational logic - * @param config the stream configs + * @param config the Kafka Streams configuration */ public KafkaStreams(final TopologyBuilder builder, final StreamsConfig config) { this(builder, config, new DefaultKafkaClientSupplier()); } /** - * Construct the stream instance. + * Create a {@code KafkaStreams} instance. * * @param builder the processor topology builder specifying the computational logic - * @param config the stream configs - * @param clientSupplier the kafka clients supplier which provides underlying producer and consumer clients - * for this {@link KafkaStreams} instance + * @param config the Kafka Streams configuration + * @param clientSupplier the Kafka clients supplier which provides underlying producer and consumer clients + * for the new {@code KafkaStreams} instance */ - public KafkaStreams(final TopologyBuilder builder, final StreamsConfig config, final KafkaClientSupplier clientSupplier) { + public KafkaStreams(final TopologyBuilder builder, + final StreamsConfig config, + final KafkaClientSupplier clientSupplier) { // create the metrics final Time time = Time.SYSTEM; @@ -289,6 +316,8 @@ public KafkaStreams(final TopologyBuilder builder, final StreamsConfig config, f if (clientId.length() <= 0) clientId = applicationId + "-" + processId; + this.logPrefix = String.format("stream-client [%s]", clientId); + final List reporters = config.getConfiguredInstances(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG, MetricsReporter.class); reporters.add(new JmxReporter(JMX_PREFIX)); @@ -308,7 +337,7 @@ public KafkaStreams(final TopologyBuilder builder, final StreamsConfig config, f final ProcessorTopology globalTaskTopology = builder.buildGlobalStateTopology(); if (config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) < 0) { - log.warn("Negative cache size passed in. Reverting to cache size of 0 bytes."); + log.warn("{} Negative cache size passed in. Reverting to cache size of 0 bytes.", logPrefix); } final long cacheSizeBytes = Math.max(0, config.getLong(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG) / @@ -336,7 +365,7 @@ public KafkaStreams(final TopologyBuilder builder, final StreamsConfig config, f time, streamsMetadataState, cacheSizeBytes); - threads[i].setStateListener(streamStateListener); + threads[i].setStateListener(new StreamStateListener()); threadState.put(threads[i].getId(), threads[i].state()); storeProviders.add(new StreamThreadStateStoreProvider(threads[i])); } @@ -358,17 +387,43 @@ private static HostInfo parseHostInfo(final String endPoint) { return new HostInfo(host, port); } - /** - * Start the stream instance by starting all its threads. + * Check if the used brokers have version 0.10.1.x or higher. + *

    + * Note, for pre 0.10.x brokers the broker version cannot be checked and the client will hang and retry + * until it {@link StreamsConfig#REQUEST_TIMEOUT_MS_CONFIG times out}. * + * @throws StreamsException if brokers have version 0.10.0.x + */ + private void checkBrokerVersionCompatibility() throws StreamsException { + final StreamsKafkaClient client = new StreamsKafkaClient(config); + + client.checkBrokerCompatibility(); + + try { + client.close(); + } catch (final IOException e) { + log.warn("{} Could not close StreamKafkaClient.", logPrefix, e); + } + + } + + /** + * Start the {@code KafkaStreams} instance by starting all its threads. + *

    + * Note, for brokers with version {@code 0.9.x} or lower, the broker version cannot be checked. + * There will be no error and the client will hang and retry to verify the broker version until it + * {@link StreamsConfig#REQUEST_TIMEOUT_MS_CONFIG times out}. + * @throws IllegalStateException if process was already started + * @throws StreamsException if the Kafka brokers have version 0.10.0.x */ - public synchronized void start() { - log.debug("Starting Kafka Stream process"); + public synchronized void start() throws IllegalStateException, StreamsException { + log.debug("{} Starting Kafka Stream process.", logPrefix); - if (state == KafkaStreams.State.CREATED) { - setState(KafkaStreams.State.RUNNING); + if (state == State.CREATED) { + checkBrokerVersionCompatibility(); + setState(State.RUNNING); if (globalStreamThread != null) { globalStreamThread.start(); @@ -378,16 +433,14 @@ public synchronized void start() { thread.start(); } - log.info("Started Kafka Stream process"); + log.info("{} Started Kafka Stream process", logPrefix); } else { throw new IllegalStateException("Cannot start again."); } } /** - * Shutdown this stream instance by signaling all the threads to stop, - * and then wait for them to join. - * + * Shutdown this {@code KafkaStreams} instance by signaling all the threads to stop, and then wait for them to join. * This will block until all threads have stopped. */ public void close() { @@ -395,72 +448,73 @@ public void close() { } /** - * Shutdown this stream instance by signaling all the threads to stop, - * and then wait up to the timeout for the threads to join. - * - * A timeout of 0 means to wait forever + * Shutdown this {@code KafkaStreams} by signaling all the threads to stop, and then wait up to the timeout for the + * threads to join. + * A {@code timeout} of 0 means to wait forever. * - * @param timeout how long to wait for {@link StreamThread}s to shutdown - * @param timeUnit unit of time used for timeout - * @return true if all threads were successfully stopped + * @param timeout how long to wait for the threads to shutdown + * @param timeUnit unit of time used for timeout + * @return {@code true} if all threads were successfully stopped—{@code false} if the timeout was reached + * before all threads stopped */ public synchronized boolean close(final long timeout, final TimeUnit timeUnit) { - log.debug("Stopping Kafka Stream process"); + log.debug("{} Stopping Kafka Stream process.", logPrefix); if (state.isCreatedOrRunning()) { - setState(KafkaStreams.State.PENDING_SHUTDOWN); + setState(State.PENDING_SHUTDOWN); // save the current thread so that if it is a stream thread // we don't attempt to join it and cause a deadlock final Thread shutdown = new Thread(new Runnable() { @Override public void run() { - // signal the threads to stop and wait - for (final StreamThread thread : threads) { - // avoid deadlocks by stopping any further state reports - // from the thread since we're shutting down - thread.setStateListener(null); - thread.close(); - } - if (globalStreamThread != null) { - globalStreamThread.close(); - if (!globalStreamThread.stillRunning()) { - try { - globalStreamThread.join(); - } catch (InterruptedException e) { - Thread.interrupted(); - } - } - } - for (final StreamThread thread : threads) { + // signal the threads to stop and wait + for (final StreamThread thread : threads) { + // avoid deadlocks by stopping any further state reports + // from the thread since we're shutting down + thread.setStateListener(null); + thread.close(); + } + if (globalStreamThread != null) { + globalStreamThread.close(); + if (!globalStreamThread.stillRunning()) { try { - if (!thread.stillRunning()) { - thread.join(); - } - } catch (final InterruptedException ex) { + globalStreamThread.join(); + } catch (final InterruptedException e) { Thread.interrupted(); } } - - metrics.close(); - log.info("Stopped Kafka Streams process"); } + for (final StreamThread thread : threads) { + try { + if (!thread.stillRunning()) { + thread.join(); + } + } catch (final InterruptedException ex) { + Thread.interrupted(); + } + } + + metrics.close(); + log.info("{} Stopped Kafka Streams process.", logPrefix); + } }, "kafka-streams-close-thread"); shutdown.setDaemon(true); shutdown.start(); try { shutdown.join(TimeUnit.MILLISECONDS.convert(timeout, timeUnit)); - } catch (InterruptedException e) { + } catch (final InterruptedException e) { Thread.interrupted(); } - setState(KafkaStreams.State.NOT_RUNNING); + setState(State.NOT_RUNNING); return !shutdown.isAlive(); } return true; } /** - * Produces a string representation containing useful information about Kafka Streams - * Such as thread IDs, task IDs and a representation of the topology. This is useful - * in debugging scenarios. + * Produce a string representation containing useful information about this {@code KafkaStream} instance such as + * thread IDs, task IDs, and a representation of the topology DAG including {@link StateStore}s (cf. + * {@link TopologyBuilder} and {@link KStreamBuilder}). + * * @return A string representation of the Kafka Streams instance. */ @Override @@ -469,13 +523,19 @@ public String toString() { } /** - * Produces a string representation containing useful information about Kafka Streams - * such as thread IDs, task IDs and a representation of the topology starting with the given indent. This is useful - * in debugging scenarios. + * Produce a string representation containing useful information about this {@code KafkaStream} instance such as + * thread IDs, task IDs, and a representation of the topology DAG including {@link StateStore}s (cf. + * {@link TopologyBuilder} and {@link KStreamBuilder}). + * + * @param indent the top-level indent for each line * @return A string representation of the Kafka Streams instance. */ public String toString(final String indent) { - final StringBuilder sb = new StringBuilder(indent + "KafkaStreams processID:" + processId + "\n"); + final StringBuilder sb = new StringBuilder() + .append(indent) + .append("KafkaStreams processID: ") + .append(processId) + .append("\n"); for (final StreamThread thread : threads) { sb.append(thread.toString(indent + "\t")); } @@ -485,11 +545,15 @@ public String toString(final String indent) { } /** - * Cleans up local state store directory ({@code state.dir}), by deleting all data with regard to the application-id. + * Do a clean up of the local {@link StateStore} directory ({@link StreamsConfig#STATE_DIR_CONFIG}) by deleting all + * data with regard to the {@link StreamsConfig#APPLICATION_ID_CONFIG application ID}. *

    - * May only be called either before instance is started or after instance is closed. + * May only be called either before this {@code KafkaStreams} instance is {@link #start() started} or after the + * instance is {@link #close() closed}. + *

    + * Calling this method triggers a restore of local {@link StateStore}s on the next {@link #start() application start}. * - * @throws IllegalStateException if instance is currently running + * @throws IllegalStateException if the instance is currently running */ public void cleanUp() { if (state.isRunning()) { @@ -500,7 +564,8 @@ public void cleanUp() { final String stateDir = config.getString(StreamsConfig.STATE_DIR_CONFIG); final String localApplicationDir = stateDir + File.separator + appId; - log.debug("Removing local Kafka Streams application data in {} for application {}", + log.debug("{} Removing local Kafka Streams application data in {} for application {}.", + logPrefix, localApplicationDir, appId); @@ -509,9 +574,10 @@ public void cleanUp() { } /** - * Sets the handler invoked when a stream thread abruptly terminates due to an uncaught exception. + * Set the handler invoked when a {@link StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} abruptly + * terminates due to an uncaught exception. * - * @param eh the object to use as this thread's uncaught exception handler. If null then this thread has no explicit handler. + * @param eh the uncaught exception handler for all internal threads; {@code null} deletes the current handler */ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh) { for (final StreamThread thread : threads) { @@ -523,25 +589,34 @@ public void setUncaughtExceptionHandler(final Thread.UncaughtExceptionHandler eh } } - /** - * Find all of the instances of {@link StreamsMetadata} in the {@link KafkaStreams} application that this instance belongs to - * + * Find all currently running {@code KafkaStreams} instances (potentially remotely) that use the same + * {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all instances that belong to + * the same Kafka Streams application) and return {@link StreamsMetadata} for each discovered instance. + *

    * Note: this is a point in time view and it may change due to partition reassignment. - * @return collection containing all instances of {@link StreamsMetadata} in the {@link KafkaStreams} application that this instance belongs to + * + * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application */ public Collection allMetadata() { validateIsRunning(); return streamsMetadataState.getAllMetadata(); } - /** - * Find instances of {@link StreamsMetadata} that contains the given storeName - * + * Find all currently running {@code KafkaStreams} instances (potentially remotely) that + *

      + *
    • use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all + * instances that belong to the same Kafka Streams application)
    • + *
    • and that contain a {@link StateStore} with the given {@code storeName}
    • + *
    + * and return {@link StreamsMetadata} for each discovered instance. + *

    * Note: this is a point in time view and it may change due to partition reassignment. - * @param storeName the storeName to find metadata for - * @return A collection containing instances of {@link StreamsMetadata} that have the provided storeName + * + * @param storeName the {@code storeName} to find metadata for + * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances with the provide {@code storeName} of + * this application */ public Collection allMetadataForStore(final String storeName) { validateIsRunning(); @@ -549,22 +624,35 @@ public Collection allMetadataForStore(final String storeName) { } /** - * Find the {@link StreamsMetadata} instance that contains the given storeName - * and the corresponding hosted store instance contains the given key. This will use - * the {@link org.apache.kafka.streams.processor.internals.DefaultStreamPartitioner} to - * locate the partition. If a custom partitioner has been used please use - * {@link KafkaStreams#metadataForKey(String, Object, StreamPartitioner)} - * - * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, - * this method provides a way of finding which host it would exist on. + * Find the currently running {@code KafkaStreams} instance (potentially remotely) that + *

      + *
    • use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all + * instances that belong to the same Kafka Streams application)
    • + *
    • and that contain a {@link StateStore} with the given {@code storeName}
    • + *
    • and the {@link StateStore} contains the given {@code key}
    • + *
    + * and return {@link StreamsMetadata} for it. + *

    + * This will use the default Kafka Streams partitioner to locate the partition. + * If a {@link StreamPartitioner custom partitioner} has been + * {@link ProducerConfig#PARTITIONER_CLASS_CONFIG configured} via {@link StreamsConfig}, + * {@link KStream#through(StreamPartitioner, String)}, or {@link KTable#through(StreamPartitioner, String, String)}, + * or if the original {@link KTable}'s input {@link KStreamBuilder#table(String, String) topic} is partitioned + * differently, please use {@link #metadataForKey(String, Object, StreamPartitioner)}. + *

    + * Note: + *

      + *
    • this is a point in time view and it may change due to partition reassignment
    • + *
    • the key may not exist in the {@link StateStore}; this method provides a way of finding which host it + * would exist on
    • + *
    * - * Note: this is a point in time view and it may change due to partition reassignment. - * @param storeName Name of the store - * @param key Key to use to for partition - * @param keySerializer Serializer for the key - * @param key type - * @return The {@link StreamsMetadata} for the storeName and key or {@link StreamsMetadata#NOT_AVAILABLE} - * if streams is (re-)initializing + * @param storeName the {@code storeName} to find metadata for + * @param key the key to find metadata for + * @param keySerializer serializer for the key + * @param key type + * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provide {@code storeName} and + * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing */ public StreamsMetadata metadataForKey(final String storeName, final K key, @@ -574,19 +662,28 @@ public StreamsMetadata metadataForKey(final String storeName, } /** - * Find the {@link StreamsMetadata} instance that contains the given storeName - * and the corresponding hosted store instance contains the given key - * - * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, - * this method provides a way of finding which host it would exist on. + * Find the currently running {@code KafkaStreams} instance (potentially remotely) that + *
      + *
    • use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all + * instances that belong to the same Kafka Streams application)
    • + *
    • and that contain a {@link StateStore} with the given {@code storeName}
    • + *
    • and the {@link StateStore} contains the given {@code key}
    • + *
    + * and return {@link StreamsMetadata} for it. + *

    + * Note: + *

      + *
    • this is a point in time view and it may change due to partition reassignment
    • + *
    • the key may not exist in the {@link StateStore}; this method provides a way of finding which host it + * would exist on
    • + *
    * - * Note: this is a point in time view and it may change due to partition reassignment. - * @param storeName Name of the store - * @param key Key to use to for partition - * @param partitioner Partitioner for the store - * @param key type - * @return The {@link StreamsMetadata} for the storeName and key or {@link StreamsMetadata#NOT_AVAILABLE} - * if streams is (re-)initializing + * @param storeName the {@code storeName} to find metadata for + * @param key the key to find metadata for + * @param partitioner the partitioner to be use to locate the host for the key + * @param key type + * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provide {@code storeName} and + * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing */ public StreamsMetadata metadataForKey(final String storeName, final K key, @@ -595,17 +692,17 @@ public StreamsMetadata metadataForKey(final String storeName, return streamsMetadataState.getMetadataWithKey(storeName, key, partitioner); } - /** - * Get a facade wrapping the {@link org.apache.kafka.streams.processor.StateStore} instances - * with the provided storeName and accepted by {@link QueryableStoreType#accepts(StateStore)}. - * The returned object can be used to query the {@link org.apache.kafka.streams.processor.StateStore} instances - * @param storeName name of the store to find - * @param queryableStoreType accept only stores that are accepted by {@link QueryableStoreType#accepts(StateStore)} - * @param return type - * @return A facade wrapping the {@link org.apache.kafka.streams.processor.StateStore} instances - * @throws org.apache.kafka.streams.errors.InvalidStateStoreException if the streams are (re-)initializing or - * a store with storeName and queryableStoreType doesnt' exist. + * Get a facade wrapping the local {@link StateStore} instances with the provided {@code storeName} if the Store's + * type is accepted by the provided {@link QueryableStoreType#accepts(StateStore) queryableStoreType}. + * The returned object can be used to query the {@link StateStore} instances. + * + * @param storeName name of the store to find + * @param queryableStoreType accept only stores that are accepted by {@link QueryableStoreType#accepts(StateStore)} + * @param return type + * @return A facade wrapping the local {@link StateStore} instances + * @throws InvalidStateStoreException if Kafka Streams is (re-)initializing or a store with {@code storeName} and + * {@code queryableStoreType} doesnt' exist */ public T store(final String storeName, final QueryableStoreType queryableStoreType) { validateIsRunning(); @@ -614,7 +711,7 @@ public T store(final String storeName, final QueryableStoreType queryable private void validateIsRunning() { if (!state.isRunning()) { - throw new IllegalStateException("KafkaStreams is not running. State is " + state); + throw new IllegalStateException("KafkaStreams is not running. State is " + state + "."); } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java index 64b38cdfbf36c..0c1d2afbcba3c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java @@ -21,8 +21,7 @@ /** * A key-value pair defined for a single Kafka Streams record. - * If the record comes directly from a Kafka topic then its - * key / value are defined as the message key / value. + * If the record comes directly from a Kafka topic then its key/value are defined as the message key/value. * * @param Key type * @param Value type @@ -37,10 +36,10 @@ public class KeyValue { /** * Create a new key-value pair. * - * @param key the key - * @param value the value + * @param key the key + * @param value the value */ - public KeyValue(K key, V value) { + public KeyValue(final K key, final V value) { this.key = key; this.value = value; } @@ -48,22 +47,23 @@ public KeyValue(K key, V value) { /** * Create a new key-value pair. * - * @param key the key - * @param value the value - * @param the type of the key - * @param the type of the value - * @return a new key value pair + * @param key the key + * @param value the value + * @param the type of the key + * @param the type of the value + * @return a new key-value pair */ - public static KeyValue pair(K key, V value) { + public static KeyValue pair(final K key, final V value) { return new KeyValue<>(key, value); } + @Override public String toString() { return "KeyValue(" + key + ", " + value + ")"; } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (this == obj) return true; @@ -71,9 +71,9 @@ public boolean equals(Object obj) { return false; } - KeyValue other = (KeyValue) obj; - return (this.key == null ? other.key == null : this.key.equals(other.key)) - && (this.value == null ? other.value == null : this.value.equals(other.value)); + final KeyValue other = (KeyValue) obj; + return (key == null ? other.key == null : key.equals(other.key)) + && (value == null ? other.value == null : value.equals(other.value)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 956ae0b7b8290..bc73f24af1c49 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -19,6 +19,8 @@ import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.config.AbstractConfig; import org.apache.kafka.common.config.ConfigDef; @@ -29,8 +31,8 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.errors.StreamsException; -import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; +import org.apache.kafka.streams.processor.FailOnInvalidTimestamp; import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.apache.kafka.streams.processor.internals.StreamThread; @@ -43,303 +45,345 @@ import static org.apache.kafka.common.config.ConfigDef.ValidString.in; /** - * Configuration for Kafka Streams. Documentation for these configurations can be found in the Kafka documentation + * Configuration for a {@link KafkaStreams} instance. + * Can also be use to configure the Kafka Streams internal {@link KafkaConsumer} and {@link KafkaProducer}. + * To avoid consumer/producer property conflicts, you should prefix those properties using + * {@link #consumerPrefix(String)} and {@link #producerPrefix(String)}, respectively. + *

    + * Example: + *

    {@code
    + * // potentially wrong: sets "metadata.max.age.ms" to 1 minute for producer AND consumer
    + * Properties streamsProperties = new Properties();
    + * streamsProperties.put(ConsumerConfig.METADATA_MAX_AGE_CONFIG, 60000);
    + * // or
    + * streamsProperties.put(ProducerConfig.METADATA_MAX_AGE_CONFIG, 60000);
    + *
    + * // suggested:
    + * Properties streamsProperties = new Properties();
    + * // sets "metadata.max.age.ms" to 1 minute for consumer only
    + * streamsProperties.put(StreamsConfig.consumerPrefix(ConsumerConfig.METADATA_MAX_AGE_CONFIG), 60000);
    + * // sets "metadata.max.age.ms" to 1 minute for producer only
    + * streamsProperties.put(StreamsConfig.producerPrefix(ProducerConfig.METADATA_MAX_AGE_CONFIG), 60000);
    + *
    + * StreamsConfig streamsConfig = new StreamsConfig(streamsProperties);
    + * }
    + * Kafka Streams required to set at least properties {@link #APPLICATION_ID_CONFIG "application.id"} and + * {@link #BOOTSTRAP_SERVERS_CONFIG "bootstrap.servers"}. + * Furthermore, it is not allowed to enable {@link ConsumerConfig#ENABLE_AUTO_COMMIT_CONFIG "enable.auto.commit"} that + * is disabled by Kafka Streams by default. + * + * @see KafkaStreams#KafkaStreams(org.apache.kafka.streams.processor.TopologyBuilder, StreamsConfig) + * @see ConsumerConfig + * @see ProducerConfig */ public class StreamsConfig extends AbstractConfig { private static final ConfigDef CONFIG; - // Prefix used to isolate consumer configs from producer configs. + /** + * Prefix used to isolate {@link KafkaConsumer consumer} configs from {@link KafkaProducer producer} configs. + * It is recommended to use {@link #consumerPrefix(String)} to add this prefix to {@link ConsumerConfig consumer + * properties}. + */ public static final String CONSUMER_PREFIX = "consumer."; // Prefix used to isolate producer configs from consumer configs. + /** + * Prefix used to isolate {@link KafkaProducer producer} configs from {@link KafkaConsumer consumer} configs. + * It is recommended to use {@link #producerPrefix(String)} to add this prefix to {@link ProducerConfig producer + * properties}. + */ public static final String PRODUCER_PREFIX = "producer."; - /** state.dir */ + /** {@code state.dir} */ public static final String STATE_DIR_CONFIG = "state.dir"; private static final String STATE_DIR_DOC = "Directory location for state store."; - /** zookeeper.connect */ + /** + * {@code zookeeper.connect} + * @deprecated Kakfa Streams does not use Zookeeper anymore and this parameter will be ignored. + */ + @Deprecated public static final String ZOOKEEPER_CONNECT_CONFIG = "zookeeper.connect"; private static final String ZOOKEEPER_CONNECT_DOC = "Zookeeper connect string for Kafka topics management."; - /** commit.interval.ms */ + /** {@code commit.interval.ms} */ public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; private static final String COMMIT_INTERVAL_MS_DOC = "The frequency with which to save the position of the processor."; - /** poll.ms */ + /** {@code poll.ms} */ public static final String POLL_MS_CONFIG = "poll.ms"; private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; - /** num.stream.threads */ + /** {@code num.stream.threads} */ public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; - /** num.standby.replicas */ + /** {@code num.standby.replicas} */ public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas"; private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task."; - /** buffered.records.per.partition */ + /** {@code buffered.records.per.partition} */ public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; private static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "The maximum number of records to buffer per partition."; - /** state.cleanup.delay */ + /** {@code state.cleanup.delay} */ public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated."; - /** timestamp.extractor */ + /** {@code timestamp.extractor} */ public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor"; private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the TimestampExtractor interface."; - /** partition.grouper */ + /** {@code partition.grouper} */ public static final String PARTITION_GROUPER_CLASS_CONFIG = "partition.grouper"; private static final String PARTITION_GROUPER_CLASS_DOC = "Partition grouper class that implements the PartitionGrouper interface."; - /** application.id */ + /** {@code application.id} */ public static final String APPLICATION_ID_CONFIG = "application.id"; - public static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix."; + private static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix."; - /** replication.factor */ + /** {@code replication.factor} */ public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; - public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."; + private static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."; - /** key.serde */ + /** {@code key.serde} */ public static final String KEY_SERDE_CLASS_CONFIG = "key.serde"; - public static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the Serde interface."; + private static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the Serde interface."; - /** value.serde */ + /** {@code value.serde} */ public static final String VALUE_SERDE_CLASS_CONFIG = "value.serde"; - public static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the Serde interface."; + private static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the Serde interface."; - /**user.endpoint */ + /**{@code user.endpoint} */ public static final String APPLICATION_SERVER_CONFIG = "application.server"; - public static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to an embedded user defined endpoint that can be used for discovering the locations of state stores within a single KafkaStreams application"; + private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to an embedded user defined endpoint that can be used for discovering the locations of state stores within a single KafkaStreams application"; - /** metrics.sample.window.ms */ + /** {@code metrics.sample.window.ms} */ public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; - /** metrics.num.samples */ + /** {@code metrics.num.samples} */ public static final String METRICS_NUM_SAMPLES_CONFIG = CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG; - /** metrics.record.level */ + /** {@code metrics.record.level} */ public static final String METRICS_RECORDING_LEVEL_CONFIG = CommonClientConfigs.METRICS_RECORDING_LEVEL_CONFIG; - /** metric.reporters */ + /** {@code metric.reporters} */ public static final String METRIC_REPORTER_CLASSES_CONFIG = CommonClientConfigs.METRIC_REPORTER_CLASSES_CONFIG; - /** bootstrap.servers */ + /** {@code bootstrap.servers} */ public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; - /** client.id */ + /** {@code client.id} */ public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; - /** rocksdb.config.setter */ + /** {@code rocksdb.config.setter} */ public static final String ROCKSDB_CONFIG_SETTER_CLASS_CONFIG = "rocksdb.config.setter"; - public static final String ROCKSDB_CONFIG_SETTER_CLASS_DOC = "A Rocks DB config setter class that implements the RocksDBConfigSetter interface"; + private static final String ROCKSDB_CONFIG_SETTER_CLASS_DOC = "A Rocks DB config setter class that implements the RocksDBConfigSetter interface"; - /** windowstore.changelog.additional.retention.ms */ + /** {@code windowstore.changelog.additional.retention.ms} */ public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG = "windowstore.changelog.additional.retention.ms"; - public static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day"; + private static final String WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC = "Added to a windows maintainMs to ensure data is not deleted from the log prematurely. Allows for clock drift. Default is 1 day"; - /** cache.max.bytes.buffering */ + /** {@code cache.max.bytes.buffering} */ public static final String CACHE_MAX_BYTES_BUFFERING_CONFIG = "cache.max.bytes.buffering"; - public static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; + private static final String CACHE_MAX_BYTES_BUFFERING_DOC = "Maximum number of memory bytes to be used for buffering across all threads"; public static final String SECURITY_PROTOCOL_CONFIG = CommonClientConfigs.SECURITY_PROTOCOL_CONFIG; - public static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC; + private static final String SECURITY_PROTOCOL_DOC = CommonClientConfigs.SECURITY_PROTOCOL_DOC; public static final String DEFAULT_SECURITY_PROTOCOL = CommonClientConfigs.DEFAULT_SECURITY_PROTOCOL; public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; - public static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC; + private static final String CONNECTIONS_MAX_IDLE_MS_DOC = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_DOC; public static final String RETRY_BACKOFF_MS_CONFIG = CommonClientConfigs.RETRY_BACKOFF_MS_CONFIG; - public static final String RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; + private static final String RETRY_BACKOFF_MS_DOC = CommonClientConfigs.RETRY_BACKOFF_MS_DOC; public static final String METADATA_MAX_AGE_CONFIG = CommonClientConfigs.METADATA_MAX_AGE_CONFIG; - public static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; + private static final String METADATA_MAX_AGE_DOC = CommonClientConfigs.METADATA_MAX_AGE_DOC; public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; - public static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC; + private static final String RECONNECT_BACKOFF_MS_DOC = CommonClientConfigs.RECONNECT_BACKOFF_MS_DOC; public static final String SEND_BUFFER_CONFIG = CommonClientConfigs.SEND_BUFFER_CONFIG; - public static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC; + private static final String SEND_BUFFER_DOC = CommonClientConfigs.SEND_BUFFER_DOC; public static final String RECEIVE_BUFFER_CONFIG = CommonClientConfigs.RECEIVE_BUFFER_CONFIG; - public static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC; + private static final String RECEIVE_BUFFER_DOC = CommonClientConfigs.RECEIVE_BUFFER_DOC; public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; - public static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; + private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; static { - CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value - Type.STRING, - Importance.HIGH, - StreamsConfig.APPLICATION_ID_DOC) - .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value - Type.LIST, - Importance.HIGH, - CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) - .define(CLIENT_ID_CONFIG, - Type.STRING, - "", - Importance.HIGH, - CommonClientConfigs.CLIENT_ID_DOC) - .define(ZOOKEEPER_CONNECT_CONFIG, - Type.STRING, - "", - Importance.HIGH, - StreamsConfig.ZOOKEEPER_CONNECT_DOC) - .define(STATE_DIR_CONFIG, - Type.STRING, - "/tmp/kafka-streams", - Importance.MEDIUM, - STATE_DIR_DOC) - .define(REPLICATION_FACTOR_CONFIG, - Type.INT, - 1, - Importance.MEDIUM, - REPLICATION_FACTOR_DOC) - .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, - Type.CLASS, - FailOnInvalidTimestamp.class.getName(), - Importance.MEDIUM, - TIMESTAMP_EXTRACTOR_CLASS_DOC) - .define(PARTITION_GROUPER_CLASS_CONFIG, - Type.CLASS, - DefaultPartitionGrouper.class.getName(), - Importance.MEDIUM, - PARTITION_GROUPER_CLASS_DOC) - .define(KEY_SERDE_CLASS_CONFIG, - Type.CLASS, - Serdes.ByteArraySerde.class.getName(), - Importance.MEDIUM, - KEY_SERDE_CLASS_DOC) - .define(VALUE_SERDE_CLASS_CONFIG, - Type.CLASS, - Serdes.ByteArraySerde.class.getName(), - Importance.MEDIUM, - VALUE_SERDE_CLASS_DOC) - .define(COMMIT_INTERVAL_MS_CONFIG, - Type.LONG, - 30000, - Importance.LOW, - COMMIT_INTERVAL_MS_DOC) - .define(POLL_MS_CONFIG, - Type.LONG, - 100, - Importance.LOW, - POLL_MS_DOC) - .define(NUM_STREAM_THREADS_CONFIG, - Type.INT, - 1, - Importance.LOW, - NUM_STREAM_THREADS_DOC) - .define(NUM_STANDBY_REPLICAS_CONFIG, - Type.INT, - 0, - Importance.LOW, - NUM_STANDBY_REPLICAS_DOC) - .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, - Type.INT, - 1000, - Importance.LOW, - BUFFERED_RECORDS_PER_PARTITION_DOC) - .define(STATE_CLEANUP_DELAY_MS_CONFIG, - Type.LONG, - 60000, - Importance.LOW, - STATE_CLEANUP_DELAY_MS_DOC) - .define(METRIC_REPORTER_CLASSES_CONFIG, - Type.LIST, - "", - Importance.LOW, - CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) - .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, - Type.LONG, - 30000, - atLeast(0), - Importance.LOW, - CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) - .define(METRICS_NUM_SAMPLES_CONFIG, - Type.INT, - 2, - atLeast(1), - Importance.LOW, - CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) - .define(METRICS_RECORDING_LEVEL_CONFIG, - Type.STRING, - Sensor.RecordingLevel.INFO.toString(), - in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString()), - Importance.LOW, - CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) - .define(APPLICATION_SERVER_CONFIG, - Type.STRING, - "", - Importance.LOW, - APPLICATION_SERVER_DOC) - .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, - Type.CLASS, - null, - Importance.LOW, - ROCKSDB_CONFIG_SETTER_CLASS_DOC) - .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, - Type.LONG, - 24 * 60 * 60 * 1000, - Importance.MEDIUM, - WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) - .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, - Type.LONG, - 10 * 1024 * 1024L, - atLeast(0), - Importance.LOW, - CACHE_MAX_BYTES_BUFFERING_DOC) - .define(SECURITY_PROTOCOL_CONFIG, - Type.STRING, - DEFAULT_SECURITY_PROTOCOL, - Importance.MEDIUM, - SECURITY_PROTOCOL_DOC) - .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, - ConfigDef.Type.LONG, - 9 * 60 * 1000, - ConfigDef.Importance.MEDIUM, - CONNECTIONS_MAX_IDLE_MS_DOC) - .define(RETRY_BACKOFF_MS_CONFIG, - ConfigDef.Type.LONG, - 100L, - atLeast(0L), - ConfigDef.Importance.LOW, - RETRY_BACKOFF_MS_DOC) - .define(METADATA_MAX_AGE_CONFIG, - ConfigDef.Type.LONG, - 5 * 60 * 1000, - atLeast(0), - ConfigDef.Importance.LOW, - METADATA_MAX_AGE_DOC) - .define(RECONNECT_BACKOFF_MS_CONFIG, - ConfigDef.Type.LONG, - 50L, - atLeast(0L), - ConfigDef.Importance.LOW, - RECONNECT_BACKOFF_MS_DOC) - .define(SEND_BUFFER_CONFIG, - ConfigDef.Type.INT, - 128 * 1024, - atLeast(0), - ConfigDef.Importance.MEDIUM, - SEND_BUFFER_DOC) - .define(RECEIVE_BUFFER_CONFIG, - ConfigDef.Type.INT, - 32 * 1024, - atLeast(0), - ConfigDef.Importance.MEDIUM, - RECEIVE_BUFFER_DOC) - .define(REQUEST_TIMEOUT_MS_CONFIG, - ConfigDef.Type.INT, - 40 * 1000, - atLeast(0), - ConfigDef.Importance.MEDIUM, - REQUEST_TIMEOUT_MS_DOC); + CONFIG = new ConfigDef() + .define(APPLICATION_ID_CONFIG, // required with no default value + Type.STRING, + Importance.HIGH, + APPLICATION_ID_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, // required with no default value + Type.LIST, + Importance.HIGH, + CommonClientConfigs.BOOTSTRAP_SERVERS_DOC) + .define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.HIGH, + CommonClientConfigs.CLIENT_ID_DOC) + .define(ZOOKEEPER_CONNECT_CONFIG, + Type.STRING, + "", + Importance.HIGH, + ZOOKEEPER_CONNECT_DOC) + .define(STATE_DIR_CONFIG, + Type.STRING, + "/tmp/kafka-streams", + Importance.MEDIUM, + STATE_DIR_DOC) + .define(REPLICATION_FACTOR_CONFIG, + Type.INT, + 1, + Importance.MEDIUM, + REPLICATION_FACTOR_DOC) + .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + FailOnInvalidTimestamp.class.getName(), + Importance.MEDIUM, + TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(PARTITION_GROUPER_CLASS_CONFIG, + Type.CLASS, + DefaultPartitionGrouper.class.getName(), + Importance.MEDIUM, + PARTITION_GROUPER_CLASS_DOC) + .define(KEY_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + KEY_SERDE_CLASS_DOC) + .define(VALUE_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + VALUE_SERDE_CLASS_DOC) + .define(COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 30000, + Importance.LOW, + COMMIT_INTERVAL_MS_DOC) + .define(POLL_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + POLL_MS_DOC) + .define(NUM_STREAM_THREADS_CONFIG, + Type.INT, + 1, + Importance.LOW, + NUM_STREAM_THREADS_DOC) + .define(NUM_STANDBY_REPLICAS_CONFIG, + Type.INT, + 0, + Importance.LOW, + NUM_STANDBY_REPLICAS_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(STATE_CLEANUP_DELAY_MS_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) + .define(METRIC_REPORTER_CLASSES_CONFIG, + Type.LIST, + "", + Importance.LOW, + CommonClientConfigs.METRIC_REPORTER_CLASSES_DOC) + .define(METRICS_SAMPLE_WINDOW_MS_CONFIG, + Type.LONG, + 30000, + atLeast(0), + Importance.LOW, + CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_DOC) + .define(METRICS_NUM_SAMPLES_CONFIG, + Type.INT, + 2, + atLeast(1), + Importance.LOW, + CommonClientConfigs.METRICS_NUM_SAMPLES_DOC) + .define(METRICS_RECORDING_LEVEL_CONFIG, + Type.STRING, + Sensor.RecordingLevel.INFO.toString(), + in(Sensor.RecordingLevel.INFO.toString(), Sensor.RecordingLevel.DEBUG.toString()), + Importance.LOW, + CommonClientConfigs.METRICS_RECORDING_LEVEL_DOC) + .define(APPLICATION_SERVER_CONFIG, + Type.STRING, + "", + Importance.LOW, + APPLICATION_SERVER_DOC) + .define(ROCKSDB_CONFIG_SETTER_CLASS_CONFIG, + Type.CLASS, + null, + Importance.LOW, + ROCKSDB_CONFIG_SETTER_CLASS_DOC) + .define(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, + Type.LONG, + 24 * 60 * 60 * 1000, + Importance.MEDIUM, + WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_DOC) + .define(CACHE_MAX_BYTES_BUFFERING_CONFIG, + Type.LONG, + 10 * 1024 * 1024L, + atLeast(0), + Importance.LOW, + CACHE_MAX_BYTES_BUFFERING_DOC) + .define(SECURITY_PROTOCOL_CONFIG, + Type.STRING, + DEFAULT_SECURITY_PROTOCOL, + Importance.MEDIUM, + SECURITY_PROTOCOL_DOC) + .define(CONNECTIONS_MAX_IDLE_MS_CONFIG, + ConfigDef.Type.LONG, + 9 * 60 * 1000, + ConfigDef.Importance.MEDIUM, + CONNECTIONS_MAX_IDLE_MS_DOC) + .define(RETRY_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 100L, + atLeast(0L), + ConfigDef.Importance.LOW, + RETRY_BACKOFF_MS_DOC) + .define(METADATA_MAX_AGE_CONFIG, + ConfigDef.Type.LONG, + 5 * 60 * 1000, + atLeast(0), + ConfigDef.Importance.LOW, + METADATA_MAX_AGE_DOC) + .define(RECONNECT_BACKOFF_MS_CONFIG, + ConfigDef.Type.LONG, + 50L, + atLeast(0L), + ConfigDef.Importance.LOW, + RECONNECT_BACKOFF_MS_DOC) + .define(SEND_BUFFER_CONFIG, + ConfigDef.Type.INT, + 128 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + SEND_BUFFER_DOC) + .define(RECEIVE_BUFFER_CONFIG, + ConfigDef.Type.INT, + 32 * 1024, + atLeast(0), + ConfigDef.Importance.MEDIUM, + RECEIVE_BUFFER_DOC) + .define(REQUEST_TIMEOUT_MS_CONFIG, + ConfigDef.Type.INT, + 40 * 1000, + atLeast(0), + ConfigDef.Importance.MEDIUM, + REQUEST_TIMEOUT_MS_DOC); } // this is the list of configs for underlying clients @@ -347,8 +391,9 @@ public class StreamsConfig extends AbstractConfig { private static final Map PRODUCER_DEFAULT_OVERRIDES; static { - Map tempProducerDefaultOverrides = new HashMap<>(); + final Map tempProducerDefaultOverrides = new HashMap<>(); tempProducerDefaultOverrides.put(ProducerConfig.LINGER_MS_CONFIG, "100"); + tempProducerDefaultOverrides.put(ProducerConfig.RETRIES_CONFIG, 10); PRODUCER_DEFAULT_OVERRIDES = Collections.unmodifiableMap(tempProducerDefaultOverrides); } @@ -356,10 +401,17 @@ public class StreamsConfig extends AbstractConfig { private static final Map CONSUMER_DEFAULT_OVERRIDES; static { - Map tempConsumerDefaultOverrides = new HashMap<>(); + final Map tempConsumerDefaultOverrides = new HashMap<>(); tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, "1000"); tempConsumerDefaultOverrides.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); tempConsumerDefaultOverrides.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + // MAX_POLL_INTERVAL_MS_CONFIG needs to be large for streams to handle cases when + // streams is recovering data from state stores. We may set it to Integer.MAX_VALUE since + // the streams code itself catches most exceptions and acts accordingly without needing + // this timeout. Note however that deadlocks are not detected (by definition) so we + // are losing the ability to detect them by setting this value to large. Hopefully + // deadlocks happen very rarely or never. + tempConsumerDefaultOverrides.put(ConsumerConfig.MAX_POLL_INTERVAL_MS_CONFIG, Integer.toString(Integer.MAX_VALUE)); CONSUMER_DEFAULT_OVERRIDES = Collections.unmodifiableMap(tempConsumerDefaultOverrides); } @@ -369,153 +421,177 @@ public static class InternalConfig { } /** - * Prefix a property with {@link StreamsConfig#CONSUMER_PREFIX}. This is used to isolate consumer configs - * from producer configs - * @param consumerProp - * @return CONSUMER_PREFIX + consumerProp + * Prefix a property with {@link #CONSUMER_PREFIX}. This is used to isolate {@link ConsumerConfig consumer configs} + * from {@link ProducerConfig producer configs}. + * + * @param consumerProp the consumer property to be masked + * @return {@link #CONSUMER_PREFIX} + {@code consumerProp} */ public static String consumerPrefix(final String consumerProp) { return CONSUMER_PREFIX + consumerProp; } /** - * Prefix a property with {@link StreamsConfig#PRODUCER_PREFIX}. This is used to isolate producer configs - * from consumer configs - * @param producerProp - * @return PRODUCER_PREFIX + consumerProp + * Prefix a property with {@link #PRODUCER_PREFIX}. This is used to isolate {@link ProducerConfig producer configs} + * from {@link ConsumerConfig consumer configs}. + * + * @param producerProp the producer property to be masked + * @return PRODUCER_PREFIX + {@code producerProp} */ public static String producerPrefix(final String producerProp) { return PRODUCER_PREFIX + producerProp; } - public StreamsConfig(Map props) { - super(CONFIG, props); + /** + * Return a copy of the config definition. + * + * @return a copy of the config definition + */ + public static ConfigDef configDef() { + return new ConfigDef(CONFIG); } /** - * Get the configs specific to the Consumer. Properties using the prefix {@link StreamsConfig#CONSUMER_PREFIX} - * will be used in favor over their non-prefixed versions except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} - * where we always use the non-prefixed version as we only support reading/writing from/to the same Kafka Cluster - * @param streamThread the {@link StreamThread} creating a consumer - * @param groupId consumer groupId - * @param clientId clientId - * @return Map of the Consumer configuration. - * @throws ConfigException + * Create a new {@code StreamsConfig} using the given properties. + * + * @param props properties that specify Kafka Streams and internal consumer/producer configuration */ - public Map getConsumerConfigs(StreamThread streamThread, String groupId, String clientId) throws ConfigException { - - final Map consumerProps = new HashMap<>(CONSUMER_DEFAULT_OVERRIDES); + public StreamsConfig(final Map props) { + super(CONFIG, props); + } + private Map getCommonConsumerConfigs() throws ConfigException { final Map clientProvidedProps = getClientPropsWithPrefix(CONSUMER_PREFIX, ConsumerConfig.configNames()); // disable auto commit and throw exception if there is user overridden values, // this is necessary for streams commit semantics if (clientProvidedProps.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { throw new ConfigException("Unexpected user-specified consumer config " + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG - + ", as the streams client will always turn off auto committing."); + + ", as the streams client will always turn off auto committing."); } + final Map consumerProps = new HashMap<>(CONSUMER_DEFAULT_OVERRIDES); consumerProps.putAll(clientProvidedProps); // bootstrap.servers should be from StreamsConfig - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.originals().get(BOOTSTRAP_SERVERS_CONFIG)); + consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, originals().get(BOOTSTRAP_SERVERS_CONFIG)); + // remove deprecate ZK config + consumerProps.remove(ZOOKEEPER_CONNECT_CONFIG); + + return consumerProps; + } + + /** + * Get the configs to the {@link KafkaConsumer consumer}. + * Properties using the prefix {@link #CONSUMER_PREFIX} will be used in favor over their non-prefixed versions + * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed + * version as we only support reading/writing from/to the same Kafka Cluster. + * + * @param streamThread the {@link StreamThread} creating a consumer + * @param groupId consumer groupId + * @param clientId clientId + * @return Map of the consumer configuration. + * @throws ConfigException if {@code "enable.auto.commit"} was set to {@code false} by the user + */ + public Map getConsumerConfigs(final StreamThread streamThread, + final String groupId, + final String clientId) throws ConfigException { + final Map consumerProps = getCommonConsumerConfigs(); + // add client id with stream client id prefix, and group id consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); consumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-consumer"); // add configs required for stream partition assignor - consumerProps.put(StreamsConfig.InternalConfig.STREAM_THREAD_INSTANCE, streamThread); - consumerProps.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, getInt(REPLICATION_FACTOR_CONFIG)); - consumerProps.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG)); + consumerProps.put(InternalConfig.STREAM_THREAD_INSTANCE, streamThread); + consumerProps.put(REPLICATION_FACTOR_CONFIG, getInt(REPLICATION_FACTOR_CONFIG)); + consumerProps.put(NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG)); consumerProps.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, StreamPartitionAssignor.class.getName()); - consumerProps.put(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, getLong(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); - if (!getString(ZOOKEEPER_CONNECT_CONFIG).equals("")) { - consumerProps.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, getString(ZOOKEEPER_CONNECT_CONFIG)); - } + consumerProps.put(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG, getLong(WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG)); consumerProps.put(APPLICATION_SERVER_CONFIG, getString(APPLICATION_SERVER_CONFIG)); + return consumerProps; } - /** - * Get the consumer config for the restore-consumer. Properties using the prefix {@link StreamsConfig#CONSUMER_PREFIX} - * will be used in favor over their non-prefixed versions except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} - * where we always use the non-prefixed version as we only support reading/writing from/to the same Kafka Cluster - * @param clientId clientId - * @return Map of the Consumer configuration - * @throws ConfigException + * Get the configs for the {@link KafkaConsumer restore-consumer}. + * Properties using the prefix {@link #CONSUMER_PREFIX} will be used in favor over their non-prefixed versions + * except in the case of {@link ConsumerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed + * version as we only support reading/writing from/to the same Kafka Cluster. + * + * @param clientId clientId + * @return Map of the consumer configuration. + * @throws ConfigException if {@code "enable.auto.commit"} was set to {@code false} by the user */ - public Map getRestoreConsumerConfigs(String clientId) throws ConfigException { - Map consumerProps = new HashMap<>(CONSUMER_DEFAULT_OVERRIDES); - - final Map clientProvidedProps = getClientPropsWithPrefix(CONSUMER_PREFIX, ConsumerConfig.configNames()); - - // disable auto commit and throw exception if there is user overridden values, - // this is necessary for streams commit semantics - if (clientProvidedProps.containsKey(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG)) { - throw new ConfigException("Unexpected user-specified consumer config " + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG - + ", as the streams client will always turn off auto committing."); - } - - consumerProps.putAll(clientProvidedProps); - - // bootstrap.servers should be from StreamsConfig - consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, this.originals().get(BOOTSTRAP_SERVERS_CONFIG)); + public Map getRestoreConsumerConfigs(final String clientId) throws ConfigException { + final Map consumerProps = getCommonConsumerConfigs(); // no need to set group id for a restore consumer consumerProps.remove(ConsumerConfig.GROUP_ID_CONFIG); - // add client id with stream client id prefix consumerProps.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-restore-consumer"); return consumerProps; } - /** - * Get the configs for the Producer. Properties using the prefix {@link StreamsConfig#PRODUCER_PREFIX} - * will be used in favor over their non-prefixed versions except in the case of {@link ProducerConfig#BOOTSTRAP_SERVERS_CONFIG} - * where we always use the non-prefixed version as we only support reading/writing from/to the same Kafka Cluster - * @param clientId clientId - * @return Map of the Consumer configuration - * @throws ConfigException + * Get the configs for the {@link KafkaProducer producer}. + * Properties using the prefix {@link #PRODUCER_PREFIX} will be used in favor over their non-prefixed versions + * except in the case of {@link ProducerConfig#BOOTSTRAP_SERVERS_CONFIG} where we always use the non-prefixed + * version as we only support reading/writing from/to the same Kafka Cluster. + * + * @param clientId clientId + * @return Map of the producer configuration. */ - public Map getProducerConfigs(String clientId) { + public Map getProducerConfigs(final String clientId) { // generate producer configs from original properties and overridden maps final Map props = new HashMap<>(PRODUCER_DEFAULT_OVERRIDES); props.putAll(getClientPropsWithPrefix(PRODUCER_PREFIX, ProducerConfig.configNames())); - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, this.originals().get(BOOTSTRAP_SERVERS_CONFIG)); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, originals().get(BOOTSTRAP_SERVERS_CONFIG)); // add client id with stream client id prefix props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-producer"); return props; } - private Map getClientPropsWithPrefix(final String prefix, final Set configNames) { + private Map getClientPropsWithPrefix(final String prefix, + final Set configNames) { final Map props = clientProps(configNames, originals()); - props.putAll(this.originalsWithPrefix(prefix)); + props.putAll(originalsWithPrefix(prefix)); return props; } + /** + * Return an {@link Serde#configure(Map, boolean) configured} instance of {@link #KEY_SERDE_CLASS_CONFIG key Serde + * class}. + * + * @return an configured instance of key Serde class + */ public Serde keySerde() { try { - Serde serde = getConfiguredInstance(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serde.class); + final Serde serde = getConfiguredInstance(KEY_SERDE_CLASS_CONFIG, Serde.class); serde.configure(originals(), true); return serde; - } catch (Exception e) { - throw new StreamsException(String.format("Failed to configure key serde %s", get(StreamsConfig.KEY_SERDE_CLASS_CONFIG)), e); + } catch (final Exception e) { + throw new StreamsException(String.format("Failed to configure key serde %s", get(KEY_SERDE_CLASS_CONFIG)), e); } } + /** + * Return an {@link Serde#configure(Map, boolean) configured} instance of {@link #VALUE_SERDE_CLASS_CONFIG value + * Serde class}. + * + * @return an configured instance of value Serde class + */ public Serde valueSerde() { try { - Serde serde = getConfiguredInstance(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serde.class); + final Serde serde = getConfiguredInstance(VALUE_SERDE_CLASS_CONFIG, Serde.class); serde.configure(originals(), false); return serde; - } catch (Exception e) { - throw new StreamsException(String.format("Failed to configure value serde %s", get(StreamsConfig.VALUE_SERDE_CLASS_CONFIG)), e); + } catch (final Exception e) { + throw new StreamsException(String.format("Failed to configure value serde %s", get(VALUE_SERDE_CLASS_CONFIG)), e); } } @@ -523,14 +599,15 @@ public Serde valueSerde() { * Override any client properties in the original configs with overrides * * @param configNames The given set of configuration names. - * @param originals The original configs to be filtered. + * @param originals The original configs to be filtered. * @return client config with any overrides */ - private Map clientProps(Set configNames, Map originals) { + private Map clientProps(final Set configNames, + final Map originals) { // iterate all client config names, filter out non-client configs from the original // property map and use the overridden values when they are not specified by users - Map parsed = new HashMap<>(); - for (String configName: configNames) { + final Map parsed = new HashMap<>(); + for (final String configName: configNames) { if (originals.containsKey(configName)) { parsed.put(configName, originals.get(configName)); } @@ -539,7 +616,7 @@ private Map clientProps(Set configNames, Map metrics(); /** - * Add a latency sensor and default associated metrics. Metrics include both latency ones - * (average and max latency) and throughput ones (operations/time unit). + * Add a latency and throughput sensor for a specific operation, which will include the following sensors: + *
      + *
    1. average latency
    2. + *
    3. max latency
    4. + *
    5. throughput (num.operations / time unit)
    6. + *
    + * Also create a parent sensor with the same metrics that aggregates all entities with the same operation under the + * same scope if it has not been created. * - * @param scopeName Name of the scope, could be the type of the state store, etc. - * @param entityName Name of the entity, could be the name of the state store instance, etc. - * @param recordingLevel The recording level (e.g., INFO or DEBUG) for this sensor. - * @param operationName Name of the operation, could be get / put / delete / etc. - * @param tags Additional tags of the sensor. + * @param scopeName name of the scope, could be the type of the state store, etc. + * @param entityName name of the entity, could be the name of the state store instance, etc. + * @param operationName name of the operation, could be get / put / delete / etc. + * @param recordingLevel the recording level (e.g., INFO or DEBUG) for this sensor. + * @param tags additional tags of the sensor * @return The added sensor. */ - Sensor addLatencySensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordingLevel, String... tags); + Sensor addLatencyAndThroughputSensor(final String scopeName, + final String entityName, + final String operationName, + final Sensor.RecordingLevel recordingLevel, + final String... tags); /** * Record the given latency value of the sensor. - * @param sensor sensor whose latency we are recording. + * If the passed sensor includes throughput metrics, e.g., when created by the + * {@link #addLatencyAndThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} method, then the + * throughput metrics will also be recorded from this event. + * + * @param sensor sensor whose latency we are recording. * @param startNs start of measurement time in nanoseconds. - * @param endNs end of measurement time in nanoseconds. + * @param endNs end of measurement time in nanoseconds. */ - void recordLatency(Sensor sensor, long startNs, long endNs); + void recordLatency(final Sensor sensor, + final long startNs, + final long endNs); /** - * Add a throughput sensor and default associated metrics. Metrics include throughput ones - * (operations/time unit). + * Add a throughput sensor for a specific operation: + *
      + *
    1. throughput (num.operations / time unit)
    2. + *
    + * Also create a parent sensor with the same metrics that aggregates all entities with the same operation under the + * same scope if it has not been created. + * This sensor is a strict subset of the sensors created by + * {@link #addLatencyAndThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)}. * - * @param scopeName Name of the scope, could be the type of the state store, etc. - * @param entityName Name of the entity, could be the name of the state store instance, etc. - * @param recordingLevel The recording level (e.g., INFO or DEBUG) for this sensor. - * @param operationName Name of the operation, could be get / put / delete / etc. - * @param tags Additional tags of the sensor. + * @param scopeName name of the scope, could be the type of the state store, etc. + * @param entityName name of the entity, could be the name of the state store instance, etc. + * @param operationName name of the operation, could be get / put / delete / etc. + * @param recordingLevel the recording level (e.g., INFO or DEBUG) for this sensor. + * @param tags additional tags of the sensor * @return The added sensor. */ - Sensor addThroughputSensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordingLevel, String... tags); + Sensor addThroughputSensor(final String scopeName, + final String entityName, + final String operationName, + final Sensor.RecordingLevel recordingLevel, + final String... tags); /** - * Records the throughput value of a sensor. - * @param sensor addSensor whose throughput we are recording. - * @param value throughput value. + * Record the throughput value of a sensor. + * + * @param sensor add Sensor whose throughput we are recording + * @param value throughput value */ - void recordThroughput(Sensor sensor, long value); + void recordThroughput(final Sensor sensor, + final long value); /** * Generic method to create a sensor. - * Note that for most cases it is advisable to use {@link #addThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} - * or {@link #addLatencySensor(String, String, String, Sensor.RecordingLevel, String...)} to ensure - * metric name well-formedness and conformity with the rest of the streams code base. However, - * if the above two methods are not sufficient, this method can also be used. - * @param name Name of the sensor. - * @param recordingLevel The recording level (e.g., INFO or DEBUG) for this sensor. + * Note that for most cases it is advisable to use + * {@link #addThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} + * or {@link #addLatencyAndThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} to ensure + * metric name well-formedness and conformity with the rest of the streams code base. + * However, if the above two methods are not sufficient, this method can also be used. + * + * @param name name of the sensor. + * @param recordingLevel the recording level (e.g., INFO or DEBUG) for this sensor + * @return The added sensor. */ - Sensor addSensor(String name, Sensor.RecordingLevel recordingLevel); + Sensor addSensor(final String name, + final Sensor.RecordingLevel recordingLevel); /** * Generic method to create a sensor with parent sensors. - * Note that for most cases it is advisable to use {@link #addThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} - * or {@link #addLatencySensor(String, String, String, Sensor.RecordingLevel, String...)} to ensure - * metric name well-formedness and conformity with the rest of the streams code base. However, - * if the above two methods are not sufficient, this method can also be used. - * @param name Name of the sensor. - * @param recordingLevel The recording level (e.g., INFO or DEBUG) for this sensor. + * Note that for most cases it is advisable to use + * {@link #addThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} + * or {@link #addLatencyAndThroughputSensor(String, String, String, Sensor.RecordingLevel, String...)} to ensure + * metric name well-formedness and conformity with the rest of the streams code base. + * However, if the above two methods are not sufficient, this method can also be used. + * + * @param name name of the sensor + * @param recordingLevel the recording level (e.g., INFO or DEBUG) for this sensor + * @return The added sensor. */ - Sensor addSensor(String name, Sensor.RecordingLevel recordingLevel, Sensor... parents); + Sensor addSensor(final String name, + final Sensor.RecordingLevel recordingLevel, + final Sensor... parents); /** * Remove a sensor. - * @param sensor Sensor to be removed. + * @param sensor sensor to be removed */ - void removeSensor(Sensor sensor); + void removeSensor(final Sensor sensor); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java index e433ea7334558..786cfa2b727d1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java @@ -20,12 +20,12 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * The {@link Aggregator} interface for aggregating values of the given key. + * The {@code Aggregator} interface for aggregating values of the given key. * This is a generalization of {@link Reducer} and allows to have different types for input value and aggregation * result. - * {@link Aggregator} is used in combination with {@link Initializer} that provides an initial aggregation value. + * {@code Aggregator} is used in combination with {@link Initializer} that provides an initial aggregation value. *

    - * {@link Aggregator} can be used to implement aggregation functions like count. + * {@code Aggregator} can be used to implement aggregation functions like count. * @param key type * @param input value type diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java index e68bf8dbe75e5..92fdf99434ab2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * The {@link ForeachAction} interface for performing an action on a {@link org.apache.kafka.streams.KeyValue key-value + * The {@code ForeachAction} interface for performing an action on a {@link org.apache.kafka.streams.KeyValue key-value * pair}. * This is a stateless record-by-record operation, i.e, {@link #apply(Object, Object)} is invoked individually for each * record of a stream. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java index 6616c908aa155..fa1a5f29be5df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/GlobalKTable.java @@ -19,40 +19,49 @@ import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; /** - * {@link GlobalKTable} is an abstraction of a changelog stream from a primary-keyed table. - * Each record in this stream is an update on the primary-keyed table with the record key as the primary key. + * {@code GlobalKTable} is an abstraction of a changelog stream from a primary-keyed table. + * Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key. *

    - * A {@link GlobalKTable} is fully replicated per {@link KafkaStreams} instance. Every partition of the underlying topic - * is consumed by each {@link GlobalKTable}, such that the full set of data is available in every {@link KafkaStreams} instance. - * This provides the ability to perform joins with {@link KStream}, and {@link KTable}, - * without having to repartition the input streams. All joins with the {@link GlobalKTable} require that a {@link KeyValueMapper} - * is provided that can map from the (key, value) of the left hand side to the key of the right hand side {@link GlobalKTable} + * {@code GlobalKTable} can only be used as right-hand side input for {@link KStream stream}-table joins. *

    - * A {@link GlobalKTable} is created via a {@link KStreamBuilder}. For example: - *

    - *     builder.globalTable("topic-name", "queryable-store-name");
    - * 
    - * all {@link GlobalKTable}s are backed by a {@link org.apache.kafka.streams.state.ReadOnlyKeyValueStore ReadOnlyKeyValueStore} - * and are therefore queryable via the interactive queries API. + * In contrast to a {@link KTable} that is partitioned over all {@link KafkaStreams} instances, a {@code GlobalKTable} + * is fully replicated per {@link KafkaStreams} instance. + * Every partition of the underlying topic is consumed by each {@code GlobalKTable}, such that the full set of data is + * available in every {@link KafkaStreams} instance. + * This provides the ability to perform joins with {@link KStream} without having to repartition the input stream. + * All joins with the {@code GlobalKTable} require that a {@link KeyValueMapper} is provided that can map from the + * {@link KeyValue} of the left hand side {@link KStream} to the key of the right hand side {@code GlobalKTable}. + *

    + * A {@code GlobalKTable} is created via a {@link KStreamBuilder}. For example: + *

    {@code
    + * builder.globalTable("topic-name", "queryable-store-name");
    + * }
    + * all {@code GlobalKTable}s are backed by a {@link ReadOnlyKeyValueStore} and are therefore queryable via the + * interactive queries API. * For example: *
    {@code
    - *     final GlobalKTable globalOne = builder.globalTable("g1", "g1-store");
    - *     final GlobalKTable globalTwo = builder.globalTable("g2", "g2-store");
    - *     ...
    - *     final KafkaStreams streams = ...;
    - *     streams.start()
    - *     ...
    - *     ReadOnlyKeyValueStore view = streams.store("g1-store", QueryableStoreTypes.keyValueStore());
    - *     view.get(key);
    + * final GlobalKTable globalOne = builder.globalTable("g1", "g1-store");
    + * final GlobalKTable globalTwo = builder.globalTable("g2", "g2-store");
    + * ...
    + * final KafkaStreams streams = ...;
    + * streams.start()
    + * ...
    + * ReadOnlyKeyValueStore view = streams.store("g1-store", QueryableStoreTypes.keyValueStore());
    + * view.get(key); // can be done on any key, as all keys are present
      *}
    - * + * Note that in contrast to {@link KTable} a {@code GlobalKTable}'s state holds a full copy if the underlying topic and + * thus, all keys can be queried locally. * * @param Type of primary keys * @param Type of value changes - * * @see KTable + * @see KStreamBuilder#globalTable(String, String) + * @see KStream#join(GlobalKTable, KeyValueMapper, ValueJoiner) + * @see KStream#leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ @InterfaceStability.Unstable public interface GlobalKTable { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java index 96a6995605f8f..bcd16b1321a1f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java @@ -20,8 +20,8 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * The {@link Initializer} interface for creating an initial value in aggregations. - * {@link Initializer} is used in combination with {@link Aggregator}. + * The {@code Initializer} interface for creating an initial value in aggregations. + * {@code Initializer} is used in combination with {@link Aggregator}. * * @param aggregate value type * @see Aggregator diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 931774356e976..3869721bdaf9d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -5,9 +5,9 @@ * 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 - * + *

    + * 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. @@ -17,22 +17,24 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.Map; /** * The window specifications used for joins. *

    - * A {@link JoinWindows} instance defines a join over two stream on the same key and a maximum time difference. + * A {@code JoinWindows} instance defines a maximum time difference for a {@link KStream#join(KStream, ValueJoiner, + * JoinWindows) join over two streams} on the same key. * In SQL-style you would express this join as - *

    + * 
    {@code
      *     SELECT * FROM stream1, stream2
      *     WHERE
      *       stream1.key = stream2.key
      *       AND
      *       stream1.ts - before <= stream2.ts AND stream2.ts <= stream1.ts + after
    - * 
    + * }
    * There are three different window configuration supported: *
      *
    • before = after = time-difference
    • @@ -42,73 +44,126 @@ * A join is symmetric in the sense, that a join specification on the first stream returns the same result record as * a join specification on the second stream with flipped before and after values. *

      - * Both values (before and after) must not result in an "inverse" window, - * i.e., lower-interval-bound must not be larger than upper-interval.bound. + * Both values (before and after) must not result in an "inverse" window, i.e., upper-interval bound cannot be smaller + * than lower-interval bound. + *

      + * {@code JoinWindows} are sliding windows, thus, they are aligned to the actual record timestamps. + * This implies, that each input record defines its own window with start and end time being relative to the record's + * timestamp. + *

      + * For time semantics, see {@link TimestampExtractor}. + * + * @see TimeWindows + * @see UnlimitedWindows + * @see SessionWindows + * @see KStream#join(KStream, ValueJoiner, JoinWindows) + * @see KStream#join(KStream, ValueJoiner, JoinWindows, org.apache.kafka.common.serialization.Serde, org.apache.kafka.common.serialization.Serde, org.apache.kafka.common.serialization.Serde) + * @see KStream#leftJoin(KStream, ValueJoiner, JoinWindows) + * @see KStream#leftJoin(KStream, ValueJoiner, JoinWindows, org.apache.kafka.common.serialization.Serde, org.apache.kafka.common.serialization.Serde, org.apache.kafka.common.serialization.Serde) + * @see KStream#outerJoin(KStream, ValueJoiner, JoinWindows) + * @see KStream#outerJoin(KStream, ValueJoiner, JoinWindows) + * @see TimestampExtractor */ -public class JoinWindows extends Windows { +@InterfaceStability.Unstable +public final class JoinWindows extends Windows { /** Maximum time difference for tuples that are before the join tuple. */ - public final long before; + public final long beforeMs; /** Maximum time difference for tuples that are after the join tuple. */ - public final long after; - - private JoinWindows(long before, long after) { - super(); + public final long afterMs; - if (before + after < 0) { - throw new IllegalArgumentException("Window interval (ie, before+after) must not be negative"); + private JoinWindows(final long beforeMs, final long afterMs) { + if (beforeMs + afterMs < 0) { + throw new IllegalArgumentException("Window interval (ie, beforeMs+afterMs) must not be negative."); } - this.after = after; - this.before = before; + this.afterMs = afterMs; + this.beforeMs = beforeMs; } /** - * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifference}. - * ({@code timeDifference} must not be negative) + * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifferenceMs}, + * i.e., the timestamp of a record from the secondary stream is max {@code timeDifferenceMs} earlier or later than + * the timestamp of the record from the primary stream. * - * @param timeDifference join window interval + * @param timeDifferenceMs join window interval in milliseconds + * @throws IllegalArgumentException if {@code timeDifferenceMs} is negative */ - public static JoinWindows of(long timeDifference) { - return new JoinWindows(timeDifference, timeDifference); + public static JoinWindows of(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(timeDifferenceMs, timeDifferenceMs); } /** - * Specifies that records of the same key are joinable if their timestamps are within - * the join window interval, and if the timestamp of a record from the secondary stream is - * earlier than or equal to the timestamp of a record from the first stream. + * Changes the start window boundary to {@code timeDifferenceMs} but keep the end window boundary as is. + * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most + * {@code timeDifferenceMs} earlier than the timestamp of the record from the primary stream. + * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "after" + * value (which would result in a negative window size). * - * @param timeDifference join window interval + * @param timeDifferenceMs relative window start time in milliseconds + * @throws IllegalArgumentException if the resulting window size is negative */ - public JoinWindows before(long timeDifference) { - return new JoinWindows(timeDifference, this.after); + public JoinWindows before(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(timeDifferenceMs, afterMs); } /** - * Specifies that records of the same key are joinable if their timestamps are within - * the join window interval, and if the timestamp of a record from the secondary stream - * is later than or equal to the timestamp of a record from the first stream. + * Changes the end window boundary to {@code timeDifferenceMs} but keep the start window boundary as is. + * Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at most + * {@code timeDifferenceMs} later than the timestamp of the record from the primary stream. + * {@code timeDifferenceMs} can be negative but it's absolute value must not be larger than current window "before" + * value (which would result in a negative window size). * - * @param timeDifference join window interval + * @param timeDifferenceMs relative window end time in milliseconds + * @throws IllegalArgumentException if the resulting window size is negative */ - public JoinWindows after(long timeDifference) { - return new JoinWindows(this.before, timeDifference); + public JoinWindows after(final long timeDifferenceMs) throws IllegalArgumentException { + return new JoinWindows(beforeMs, timeDifferenceMs); } /** - * Not supported by {@link JoinWindows}. Throws {@link UnsupportedOperationException}. + * Not supported by {@code JoinWindows}. + * Throws {@link UnsupportedOperationException}. + * + * @throws UnsupportedOperationException at every invocation */ @Override - public Map windowsFor(long timestamp) { - throw new UnsupportedOperationException("windowsFor() is not supported in JoinWindows"); + public Map windowsFor(final long timestamp) { + throw new UnsupportedOperationException("windowsFor() is not supported by JoinWindows."); } @Override public long size() { - return after + before; + return beforeMs + afterMs; + } + + /** + * @param durationMs the window retention time in milliseconds + * @return itself + * @throws IllegalArgumentException if {@code durationMs} is smaller than the window size + */ + @Override + public JoinWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < size()) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be smaller than the window size."); + } + super.until(durationMs); + return this; + } + + /** + * {@inheritDoc} + *

      + * For {@link TimeWindows} the maintain duration is at least as small as the window size. + * + * @return the window maintain duration + */ + @Override + public long maintainMs() { + return Math.max(super.maintainMs(), size()); } @Override - public final boolean equals(Object o) { + public final boolean equals(final Object o) { if (o == this) { return true; } @@ -116,14 +171,14 @@ public final boolean equals(Object o) { return false; } - JoinWindows other = (JoinWindows) o; - return this.before == other.before && this.after == other.after; + final JoinWindows other = (JoinWindows) o; + return beforeMs == other.beforeMs && afterMs == other.afterMs; } @Override public int hashCode() { - int result = (int) (before ^ (before >>> 32)); - result = 31 * result + (int) (after ^ (after >>> 32)); + int result = (int) (beforeMs ^ (beforeMs >>> 32)); + result = 31 * result + (int) (afterMs ^ (afterMs >>> 32)); return result; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java index 8e69fdb89c67e..d767f0be62087 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedStream.java @@ -17,19 +17,24 @@ import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreType; -import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.streams.state.WindowStore; /** - * {@link KGroupedStream} is an abstraction of a grouped record stream of key-value pairs. + * {@code KGroupedStream} is an abstraction of a grouped record stream of {@link KeyValue} pairs. * It is an intermediate representation of a {@link KStream} in order to apply an aggregation operation on the original * {@link KStream} records. *

      - * A {@link KGroupedStream} must be obtained from a {@link KStream} via {@link KStream#groupByKey() #groupByKey()} or - * {@link KStream#groupBy(KeyValueMapper) #groupBy(...)}. + * It is an intermediate representation after a grouping of a {@link KStream} before an aggregation is applied to the + * new partitions resulting in a {@link KTable}. + *

      + * A {@code KGroupedStream} must be obtained from a {@link KStream} via {@link KStream#groupByKey() groupByKey()} or + * {@link KStream#groupBy(KeyValueMapper) groupBy(...)}. * * @param Type of keys * @param Type of values @@ -40,7 +45,7 @@ public interface KGroupedStream { /** * Count the number of records in this stream by the grouped key. - * Records with {@code null} value are ignored. + * Records with {@code null} key or value are ignored. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * that can be queried using the provided {@code storeName}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. @@ -48,12 +53,12 @@ public interface KGroupedStream { * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to * the same key. * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // counting words
            * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      @@ -65,25 +70,33 @@ public interface KGroupedStream {
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * * @param storeName the name of the underlying {@link KTable} state store - * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest - * (rolling) count (i.e., number of records) for each key + * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that + * represent the latest (rolling) count (i.e., number of records) for each key */ KTable count(final String storeName); /** * Count the number of records in this stream by the grouped key. - * Records with {@code null} value are ignored. - * The result is written into a local {@link KeyValueStore} provided by the given {@code storeSupplier}. + * Records with {@code null} key or value are ignored. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // counting words
      @@ -95,15 +108,15 @@ public interface KGroupedStream {
            * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to
            * query the value of the key on a parallel running instance of your Kafka Streams application.
            *
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest
      -     * (rolling) count (i.e., number of records) for each key
      +     * @param storeSupplier user defined state store supplier
      +     * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that
      +     * represent the latest (rolling) count (i.e., number of records) for each key
            */
           KTable count(final StateStoreSupplier storeSupplier);
       
           /**
            * Count the number of records in this stream by the grouped key and the defined windows.
      -     * Records with {@code null} value are ignored.
      +     * Records with {@code null} key or value are ignored.
            * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
            * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
            * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
      @@ -115,12 +128,12 @@ public interface KGroupedStream {
            * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to
            * the same window and key.
            * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of
      -     * parallel running Kafka Streams instances, and the cache size.
      -     * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter
      -     * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}.
      +     * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for
      +     * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and
      +     * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}.
            * 

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // counting words
            * ReadOnlyWindowStore localWindowStore = streams.store(storeName, QueryableStoreTypes.windowStore());
      @@ -134,31 +147,39 @@ public interface KGroupedStream {
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * * @param windows the specification of the aggregation {@link Windows} * @param storeName the name of the underlying {@link KTable} state store - * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent - * the latest (rolling) count (i.e., number of records) for each key within a window + * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values + * that represent the latest (rolling) count (i.e., number of records) for each key within a window */ KTable, Long> count(final Windows windows, final String storeName); /** * Count the number of records in this stream by the grouped key and the defined windows. - * Records with {@code null} value are ignored. + * Records with {@code null} key or value are ignored. * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f. * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}). - * The result is written into a local windowed {@link KeyValueStore} provided by the given {@code storeSupplier}. + * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating + * materialized view) provided by the given {@code storeSupplier}. * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}). * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // counting words
      @@ -173,9 +194,9 @@  KTable, Long> count(final Windows windows,
            * query the value of the key on a parallel running instance of your Kafka Streams application.
            *
            * @param windows       the specification of the aggregation {@link Windows}
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent
      -     * the latest (rolling) count (i.e., number of records) for each key within a window
      +     * @param storeSupplier user defined state store supplier
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      +     * that represent the latest (rolling) count (i.e., number of records) for each key within a window
            */
            KTable, Long> count(final Windows windows,
                                                              final StateStoreSupplier storeSupplier);
      @@ -183,15 +204,22 @@  KTable, Long> count(final Windows windows,
       
           /**
            * Count the number of records in this stream by the grouped key into {@link SessionWindows}.
      -     * Records with {@code null} value are ignored.
      +     * Records with {@code null} key or value are ignored.
            * The result is written into a local {@link SessionStore} (which is basically an ever-updating
            * materialized view) that can be queried using the provided {@code storeName}.
            * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
            * 

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // counting words
      @@ -206,22 +234,29 @@  KTable, Long> count(final Windows windows,
            *
            * @param sessionWindows the specification of the aggregation {@link SessionWindows}
            * @param storeName      the name of the state store created from this operation.
      -     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
      -     *         where each table contains records with unmodified keys and values
      -     *         that represent the latest (rolling) count (i.e., number of records) for each key within that window
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      +     * that represent the latest (rolling) count (i.e., number of records) for each key within a window
            */
           KTable, Long> count(final SessionWindows sessionWindows, final String storeName);
       
           /**
            * Count the number of records in this stream by the grouped key into {@link SessionWindows}.
      -     * Records with {@code null} value are ignored.
      -     * The result is written into a local {@link SessionStore} provided by the given {@code storeSupplier}.
      +     * Records with {@code null} key or value are ignored.
      +     * The result is written into a local {@link SessionStore} (which is basically an ever-updating materialized view)
      +     * provided by the given {@code storeSupplier}.
            * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
            * 

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // counting words
      @@ -235,38 +270,37 @@  KTable, Long> count(final Windows windows,
            *
            *
            * @param sessionWindows the specification of the aggregation {@link SessionWindows}
      -     * @param storeSupplier  user defined state store supplier {@link StateStoreSupplier}
      -     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
      -     *         where each table contains records with unmodified keys and values
      -     *         that represent the latest (rolling) count (i.e., number of records) for each key within that window
      +     * @param storeSupplier  user defined state store supplier
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and {@link Long} values
      +     * that represent the latest (rolling) count (i.e., number of records) for each key within a window
            */
           KTable, Long> count(final SessionWindows sessionWindows,
                                           final StateStoreSupplier storeSupplier);
       
           /**
            * Combine the values of records in this stream by the grouped key.
      -     * Records with {@code null} value are ignored.
      +     * Records with {@code null} key or value are ignored.
            * Combining implies that the type of the aggregate result is the same as the type of the input value
            * (c.f. {@link #aggregate(Initializer, Aggregator, Serde, String)}).
            * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view)
            * that can be queried using the provided {@code storeName}.
            * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream.
            * 

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, String)} can be used to compute aggregate functions like sum, min, or max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // compute sum
            * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      @@ -278,39 +312,48 @@ KTable, Long> count(final SessionWindows sessionWindows,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * * @param reducer a {@link Reducer} that computes a new aggregate result * @param storeName the name of the underlying {@link KTable} state store - * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest - * (rolling) aggregate for each key + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ KTable reduce(final Reducer reducer, final String storeName); /** * Combine the value of records in this stream by the grouped key. - * Records with {@code null} value are ignored. + * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, StateStoreSupplier)}). - * The result is written into a local {@link KeyValueStore} provided by the given {@code storeSupplier}. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, StateStoreSupplier)} can be used to compute aggregate functions like sum, min, or * max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // compute sum
      +     * String storeName = storeSupplier.name();
            * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
            * String key = "some-key";
            * Long sumForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      @@ -319,16 +362,16 @@ KTable reduce(final Reducer reducer,
            * query the value of the key on a parallel running instance of your Kafka Streams application.
            *
            * @param reducer   a {@link Reducer} that computes a new aggregate result
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest
      -     * (rolling) aggregate for each key
      +     * @param storeSupplier user defined state store supplier
      +     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      +     * latest (rolling) aggregate for each key
            */
           KTable reduce(final Reducer reducer,
                               final StateStoreSupplier storeSupplier);
       
           /**
            * Combine the number of records in this stream by the grouped key and the defined windows.
      -     * Records with {@code null} value are ignored.
      +     * Records with {@code null} key or value are ignored.
            * Combining implies that the type of the aggregate result is the same as the type of the input value
            * (c.f. {@link #aggregate(Initializer, Aggregator, Windows, Serde, String)}).
            * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
      @@ -339,21 +382,21 @@ KTable reduce(final Reducer reducer,
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
            * 

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, Windows, String)} can be used to compute aggregate functions like sum, min, or max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // compute sum
            * ReadOnlyWindowStore localWindowStore = streams.store(storeName, QueryableStoreTypes.windowStore());
      @@ -367,15 +410,15 @@ KTable reduce(final Reducer reducer,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * * @param reducer a {@link Reducer} that computes a new aggregate result * @param windows the specification of the aggregation {@link Windows} * @param storeName the name of the state store created from this operation - * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent + * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent * the latest (rolling) aggregate for each key within a window */ KTable, V> reduce(final Reducer reducer, @@ -384,25 +427,33 @@ KTable, V> reduce(final Reducer reducer, /** * Combine the values of records in this stream by the grouped key and the defined windows. - * Records with {@code null} value are ignored. + * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, Windows, Serde, String)}). * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f. * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}). - * The result is written into a local windowed {@link KeyValueStore} provided by the given {@code storeSupplier}. + * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating + * materialized view) provided by the given {@code storeSupplier}. * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}). * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID. *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, Windows, StateStoreSupplier)} can be used to compute aggregate functions like sum, * min, or max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // compute sum
      @@ -418,8 +469,8 @@  KTable, V> reduce(final Reducer reducer,
            *
            * @param reducer       a {@link Reducer} that computes a new aggregate result
            * @param windows       the specification of the aggregation {@link Windows}
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent
      +     * @param storeSupplier user defined state store supplier
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
            * the latest (rolling) aggregate for each key within a window
            */
            KTable, V> reduce(final Reducer reducer,
      @@ -428,6 +479,7 @@  KTable, V> reduce(final Reducer reducer,
       
           /**
            * Combine values of this stream by the grouped key into {@link SessionWindows}.
      +     * Records with {@code null} key or value are ignored.
            * Combining implies that the type of the aggregate result is the same as the type of the input value
            * (c.f. {@link #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)}).
            * The result is written into a local {@link SessionStore} (which is basically an ever-updating
      @@ -436,22 +488,22 @@  KTable, V> reduce(final Reducer reducer,
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
            * 

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, SessionWindows, String)} can be used to compute aggregate functions like sum, min, * or max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // compute sum
            * ReadOnlySessionStore sessionStore = streams.store(storeName, QueryableStoreTypes.sessionStore());
      @@ -463,16 +515,15 @@  KTable, V> reduce(final Reducer reducer,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * @param reducer the instance of {@link Reducer} * @param sessionWindows the specification of the aggregation {@link SessionWindows} * @param storeName the name of the state store created from this operation - * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s - * where each table contains records with unmodified keys and values - * that represent the latest (rolling) aggregate for each key within that window + * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent + * the latest (rolling) aggregate for each key within a window */ KTable, V> reduce(final Reducer reducer, final SessionWindows sessionWindows, @@ -480,31 +531,35 @@ KTable, V> reduce(final Reducer reducer, /** * Combine values of this stream by the grouped key into {@link SessionWindows}. + * Records with {@code null} key or value are ignored. * Combining implies that the type of the aggregate result is the same as the type of the input value * (c.f. {@link #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)}). - * The result is written into a local {@link SessionStore} provided by the given {@code storeSupplier}. + * The result is written into a local {@link SessionStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}). * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID. *

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Reducer} is applied for each input record and computes a new aggregate using the current * aggregate and the record's value. - * If there is no current aggregate the {@link Reducer} is not applied an the new aggregate will be the record's + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's * value as-is. * Thus, {@code reduce(Reducer, SessionWindows, StateStoreSupplier)} can be used to compute aggregate functions like * sum, min, or max. *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // compute sum
      +     * Sting storeName = storeSupplier.name();
            * ReadOnlySessionStore sessionStore = streams.store(storeName, QueryableStoreTypes.sessionStore());
            * String key = "some-key";
            * KeyValueIterator, Long> sumForKeyForSession = localWindowStore.fetch(key); // key must be local (application state is shared over all running Kafka Streams instances)
      @@ -514,16 +569,15 @@ KTable, V> reduce(final Reducer reducer,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * @param reducer the instance of {@link Reducer} * @param sessionWindows the specification of the aggregation {@link SessionWindows} - * @param storeSupplier user defined state store supplier {@link StateStoreSupplier} - * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s - * where each table contains records with unmodified keys and values - * that represent the latest (rolling) aggregate for each key within that window + * @param storeSupplier user defined state store supplier + * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent + * the latest (rolling) aggregate for each key within a window */ KTable, V> reduce(final Reducer reducer, final SessionWindows sessionWindows, @@ -532,30 +586,30 @@ KTable, V> reduce(final Reducer reducer, /** * Aggregate the values of records in this stream by the grouped key. - * Records with {@code null} value are ignored. - * Aggregating is a generalization of {@link #reduce(Reducer, String) combining via reduce(...)} as it allows the - * result to have a different type than the input values. + * Records with {@code null} key or value are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, String) combining via reduce(...)} as it, for example, + * allows the result to have a different type than the input values. * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) * that can be queried using the provided {@code storeName}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Initializer} is applied once directly before the first input record is processed to * provide an initial intermediate aggregation result that is used to process the first record. * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, Serde, String)} can be used to compute aggregate functions like - * count (c.f. {@link #count(String)}) + * count (c.f. {@link #count(String)}). + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // some aggregation on value type double
            * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      @@ -567,8 +621,8 @@ KTable, V> reduce(final Reducer reducer,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * @@ -578,8 +632,8 @@ KTable, V> reduce(final Reducer reducer, * if not specified the default serdes defined in the configs will be used * @param storeName the name of the state store created from this operation * @param the value type of the resulting {@link KTable} - * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest - * (rolling) aggregate for each key + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ KTable aggregate(final Initializer initializer, final Aggregator aggregator, @@ -588,10 +642,11 @@ KTable aggregate(final Initializer initializer, /** * Aggregate the values of records in this stream by the grouped key. - * Records with {@code null} value are ignored. - * Aggregating is a generalization of {@link #reduce(Reducer, StateStoreSupplier)} combining via reduce(...)} as it - * allows the result to have a different type than the input values. - * The result is written into a local {@link KeyValueStore} provided by the given {@code storeSupplier}. + * Records with {@code null} key or value are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, StateStoreSupplier) combining via reduce(...)} as it, + * for example, allows the result to have a different type than the input values. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. *

      * The specified {@link Initializer} is applied once directly before the first input record is processed to @@ -600,13 +655,21 @@ KTable aggregate(final Initializer initializer, * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, StateStoreSupplier)} can be used to compute aggregate functions - * like count (c.f. {@link #count(String)}) + * like count (c.f. {@link #count(String)}). + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // some aggregation on value type double
      +     * Sting storeName = storeSupplier.name();
            * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
            * String key = "some-key";
            * Long aggForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      @@ -616,10 +679,10 @@  KTable aggregate(final Initializer initializer,
            *
            * @param initializer   an {@link Initializer} that computes an initial intermediate aggregation result
            * @param aggregator    an {@link Aggregator} that computes a new aggregate result
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      +     * @param storeSupplier user defined state store supplier
            * @param           the value type of the resulting {@link KTable}
      -     * @return a {@link KTable} that contains "update" records with unmodified keys and values that represent the latest
      -     * (rolling) aggregate for each key
      +     * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the
      +     * latest (rolling) aggregate for each key
            */
            KTable aggregate(final Initializer initializer,
                                        final Aggregator aggregator,
      @@ -627,9 +690,9 @@  KTable aggregate(final Initializer initializer,
       
           /**
            * Aggregate the values of records in this stream by the grouped key and defined windows.
      -     * Records with {@code null} value are ignored.
      -     * Aggregating is a generalization of {@link #reduce(Reducer, Windows, String)} combining via reduce(...)} as it
      -     * allows the result to have a different type than the input values.
      +     * Records with {@code null} key or value are ignored.
      +     * Aggregating is a generalization of {@link #reduce(Reducer, Windows, String) combining via reduce(...)} as it,
      +     * for example, allows the result to have a different type than the input values.
            * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f.
            * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}).
            * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating
      @@ -638,23 +701,23 @@  KTable aggregate(final Initializer initializer,
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
            * 

      - * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to - * the same key. - * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of - * parallel running Kafka Streams instances, and the cache size. - * You can configure the cache size via {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} parameter - * {@link org.apache.kafka.streams.StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG CACHE_MAX_BYTES_BUFFERING_CONFIG}. - *

      * The specified {@link Initializer} is applied once per window directly before the first input record is * processed to provide an initial intermediate aggregation result that is used to process the first record. * The specified {@link Aggregator} is applied for each input record and computes a new aggregate using the current * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, Windows, Serde, String)} can be used to compute aggregate - * functions like count (c.f. {@link #count(String)}) + * functions like count (c.f. {@link #count(String)}). + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: *

      {@code
            * KafkaStreams streams = ... // some windowed aggregation on value type double
            * ReadOnlyWindowStore localWindowStore = streams.store(storeName, QueryableStoreTypes.windowStore());
      @@ -668,8 +731,8 @@  KTable aggregate(final Initializer initializer,
            * 

      * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the * provide {@code storeName}, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * @@ -681,7 +744,7 @@ KTable aggregate(final Initializer initializer, * if not specified the default serdes defined in the configs will be used * @param the value type of the resulting {@link KTable} * @param storeName the name of the state store created from this operation - * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent + * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent * the latest (rolling) aggregate for each key within a window */ KTable, VR> aggregate(final Initializer initializer, @@ -692,12 +755,13 @@ KTable, VR> aggregate(final Initializer i /** * Aggregate the values of records in this stream by the grouped key and defined windows. - * Records with {@code null} value are ignored. - * Aggregating is a generalization of {@link #reduce(Reducer, Windows, StateStoreSupplier)} combining via - * reduce(...)} as it allows the result to have a different type than the input values. + * Records with {@code null} key or value are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, Windows, StateStoreSupplier) combining via + * reduce(...)} as it, for example, allows the result to have a different type than the input values. * The specified {@code windows} define either hopping time windows that can be overlapping or tumbling (c.f. * {@link TimeWindows}) or they define landmark windows (c.f. {@link UnlimitedWindows}). - * The result is written into a local windowed {@link KeyValueStore} provided by the given {@code storeSupplier}. + * The result is written into a local windowed {@link KeyValueStore} (which is basically an ever-updating + * materialized view) provided by the given {@code storeSupplier}. * Windows are retained until their retention time expires (c.f. {@link Windows#until(long)}). * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID. @@ -708,13 +772,20 @@ KTable, VR> aggregate(final Initializer i * aggregate (or for the very first record using the intermediate aggregation result provided via the * {@link Initializer}) and the record's value. * Thus, {@code aggregate(Initializer, Aggregator, Windows, StateStoreSupplier)} can be used to compute aggregate - * functions like count (c.f. {@link #count(String)}) TODO add more examples. + * functions like count (c.f. {@link #count(String)}). + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local windowed {@link KeyValueStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
      -     * KafkaStreams streams = ... // some windowed aggregation on value type double TODO update example
      +     * KafkaStreams streams = ... // some windowed aggregation on value type Long
            * Sting storeName = storeSupplier.name();
            * ReadOnlyWindowStore localWindowStore = streams.store(storeName, QueryableStoreTypes.windowStore());
            * String key = "some-key";
      @@ -729,8 +800,8 @@  KTable, VR> aggregate(final Initializer i
            * @param aggregator    an {@link Aggregator} that computes a new aggregate result
            * @param windows       the specification of the aggregation {@link Windows}
            * @param           the value type of the resulting {@link KTable}
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a windowed {@link KTable} that contains "update" records with unmodified keys and values that represent
      +     * @param storeSupplier user defined state store supplier
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
            * the latest (rolling) aggregate for each key within a window
            */
            KTable, VR> aggregate(final Initializer initializer,
      @@ -740,9 +811,9 @@  KTable, VR> aggregate(final Initializer i
       
           /**
            * Aggregate the values of records in this stream by the grouped key and defined {@link SessionWindows}.
      -     * Records with {@code null} value are ignored.
      -     * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String)} combining via
      -     * reduce(...)} as it allows the result to have a different type than the input values.
      +     * Records with {@code null} key or value are ignored.
      +     * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String) combining via
      +     * reduce(...)} as it, for example, allows the result to have a different type than the input values.
            * The result is written into a local {@link SessionStore} (which is basically an ever-updating
            * materialized view) that can be queried using the provided {@code storeName}.
            * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
      @@ -757,8 +828,15 @@  KTable, VR> aggregate(final Initializer i
            * Thus, {@code aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, String)} can be used to compute
            * aggregate functions like count (c.f. {@link #count(String)})
            * 

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // some windowed aggregation on value type double
      @@ -778,9 +856,8 @@  KTable, VR> aggregate(final Initializer i
            *                      if not specified the default serdes defined in the configs will be used
            * @param            the value type of the resulting {@link KTable}
            * @param storeName     the name of the state store created from this operation
      -     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
      -     *         where each table contains records with unmodified keys and values with type {@code T}
      -     *         that represent the latest (rolling) aggregate for each key within that window
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      +     * the latest (rolling) aggregate for each key within a window
            */
            KTable, T> aggregate(final Initializer initializer,
                                                final Aggregator aggregator,
      @@ -791,10 +868,11 @@  KTable, T> aggregate(final Initializer initializer,
       
           /**
            * Aggregate the values of records in this stream by the grouped key and defined {@link SessionWindows}.
      -     * Records with {@code null} value are ignored.
      -     * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String)} combining via
      -     * reduce(...)} as it allows the result to have a different type than the input values.
      -     * The result is written into a local {@link SessionStore} provided by the given {@code storeSupplier}.
      +     * Records with {@code null} key or value are ignored.
      +     * Aggregating is a generalization of {@link #reduce(Reducer, SessionWindows, String) combining via
      +     * reduce(...)} as it, for example, allows the result to have a different type than the input values.
      +     * The result is written into a local {@link SessionStore} (which is basically an ever-updating materialized view)
      +     * provided by the given {@code storeSupplier}.
            * SessionWindows are retained until their retention time expires (c.f. {@link SessionWindows#until(long)}).
            * Furthermore, updates to the store are sent downstream into a windowed {@link KTable} changelog stream, where
            * "windowed" implies that the {@link KTable} key is a combined key of the original record key and a window ID.
      @@ -805,10 +883,17 @@  KTable, T> aggregate(final Initializer initializer,
            * aggregate (or for the very first record using the intermediate aggregation result provided via the
            * {@link Initializer}) and the record's value.
            * Thus, {@code #aggregate(Initializer, Aggregator, Merger, SessionWindows, Serde, StateStoreSupplier)} can be used
      -     * to compute aggregate functions like count (c.f. {@link #count(String)})
      +     * to compute aggregate functions like count (c.f. {@link #count(String)}).
      +     * 

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same window and key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. *

      * To query the local {@link SessionStore} it must be obtained via - * {@link org.apache.kafka.streams.KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}. * Use {@link StateStoreSupplier#name()} to get the store name: *

      {@code
            * KafkaStreams streams = ... // some windowed aggregation on value type double
      @@ -827,11 +912,10 @@  KTable, T> aggregate(final Initializer initializer,
            * @param sessionWindows the specification of the aggregation {@link SessionWindows}
            * @param aggValueSerde  aggregate value serdes for materializing the aggregated table,
            *                       if not specified the default serdes defined in the configs will be used
      -     * @param storeSupplier  user defined state store supplier {@link StateStoreSupplier}
      +     * @param storeSupplier  user defined state store supplier
            * @param            the value type of the resulting {@link KTable}
      -     * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s
      -     *         where each table contains records with unmodified keys and values with type {@code T}
      -     *         that represent the latest (rolling) aggregate for each key within that window
      +     * @return a windowed {@link KTable} that contains "update" records with unmodified keys, and values that represent
      +     * the latest (rolling) aggregate for each key within a window
            */
            KTable, T> aggregate(final Initializer initializer,
                                                final Aggregator aggregator,
      diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
      index 9e23c071de6ed..53c0a127fc409 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java
      @@ -19,139 +19,488 @@
       
       import org.apache.kafka.common.annotation.InterfaceStability;
       import org.apache.kafka.common.serialization.Serde;
      +import org.apache.kafka.streams.KafkaStreams;
      +import org.apache.kafka.streams.StreamsConfig;
       import org.apache.kafka.streams.processor.StateStoreSupplier;
       import org.apache.kafka.streams.state.KeyValueStore;
      +import org.apache.kafka.streams.state.QueryableStoreType;
       
       /**
      - * {@link KGroupedTable} is an abstraction of a grouped changelog stream from a primary-keyed table,
      + * {@code KGroupedTable} is an abstraction of a re-grouped changelog stream from a primary-keyed table,
        * usually on a different grouping key than the original primary key.
        * 

      - * It is an intermediate representation after a re-grouping of a {@link KTable} before an aggregation is applied - * to the new partitions resulting in a new {@link KTable}. + * It is an intermediate representation after a re-grouping of a {@link KTable} before an aggregation is applied to the + * new partitions resulting in a new {@link KTable}. + *

      + * A {@code KGroupedTable} must be obtained from a {@link KTable} via {@link KTable#groupBy(KeyValueMapper) + * groupBy(...)}. * - * @param Type of primary keys - * @param Type of value changes + * @param Type of keys + * @param Type of values + * @see KTable */ @InterfaceStability.Unstable public interface KGroupedTable { /** - * Combine updating values of this stream by the selected key into a new instance of {@link KTable}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Count number of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) mapped} to + * the same key into a new instance of {@link KTable}. + * Records with {@code null} key are ignored. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * that can be queried using the provided {@code storeName}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param adder the instance of {@link Reducer} for addition - * @param subtractor the instance of {@link Reducer} for subtraction - * @param storeName the name of the underlying {@link KTable} state store - * @return a {@link KTable} with the same key and value types as this {@link KGroupedTable}, - * containing aggregated values for each key + * @param storeName the name of the underlying {@link KTable} state store + * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that + * represent the latest (rolling) count (i.e., number of records) for each key */ - KTable reduce(Reducer adder, - Reducer subtractor, - String storeName); + KTable count(final String storeName); /** - * Combine updating values of this stream by the selected key into a new instance of {@link KTable}. - * The resulting {@link KTable} will be materialized in a state - * store provided by the {@link StateStoreSupplier}. + * Count number of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) mapped} to + * the same key into a new instance of {@link KTable}. + * Records with {@code null} key are ignored. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * String storeName = storeSupplier.name();
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param adder the instance of {@link Reducer} for addition - * @param subtractor the instance of {@link Reducer} for subtraction - * @param storeSupplier user defined state store supplier {@link StateStoreSupplier} - * @return a {@link KTable} with the same key and value types as this {@link KGroupedTable}, - * containing aggregated values for each key + * @param storeSupplier user defined state store supplier + * @return a {@link KTable} that contains "update" records with unmodified keys and {@link Long} values that + * represent the latest (rolling) count (i.e., number of records) for each key */ - KTable reduce(Reducer adder, - Reducer subtractor, - final StateStoreSupplier storeSupplier); + KTable count(final StateStoreSupplier storeSupplier); /** - * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Combine the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) + * mapped} to the same key into a new instance of {@link KTable}. + * Records with {@code null} key are ignored. + * Combining implies that the type of the aggregate result is the same as the type of the input value + * (c.f. {@link #aggregate(Initializer, Aggregator, Aggregator, Serde, String)}). + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * that can be queried using the provided {@code storeName}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. + * The specified {@link Reducer adder} is applied for each update record and computes a new aggregate using the + * current aggregate and the record's value by adding the new record to the aggregate. + * The specified {@link Reducer substractor} is applied for each "replaced" record of the original {@link KTable} + * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" + * record from the aggregate. + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's + * value as-is. + * Thus, {@code reduce(Reducer, Reducer, String)} can be used to compute aggregate functions like sum. + * For sum, the adder and substractor would work as follows: + *

      {@code
      +     * public class SumAdder implements Reducer {
      +     *   public Integer apply(Integer currentAgg, Integer newValue) {
      +     *     return currentAgg + newValue;
      +     *   }
      +     * }
            *
      -     * @param initializer   the instance of {@link Initializer}
      -     * @param adder         the instance of {@link Aggregator} for addition
      -     * @param subtractor    the instance of {@link Aggregator} for subtraction
      -     * @param aggValueSerde value serdes for materializing the aggregated table,
      -     *                      if not specified the default serdes defined in the configs will be used
      -     * @param storeName     the name of the underlying {@link KTable} state store
      -     * @param            the value type of the aggregated {@link KTable}
      -     * @return a {@link KTable} with same key and aggregated value type {@code T},
      -     *         containing aggregated values for each key
      +     * public class SumSubtractor implements Reducer {
      +     *   public Integer apply(Integer currentAgg, Integer oldValue) {
      +     *     return currentAgg - oldValue;
      +     *   }
      +     * }
      +     * }
      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + * + * @param adder a {@link Reducer} that adds a new value to the aggregate result + * @param subtractor a {@link Reducer} that removed an old value from the aggregate result + * @param storeName the name of the underlying {@link KTable} state store + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ - KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator subtractor, - Serde aggValueSerde, - String storeName); + KTable reduce(final Reducer adder, + final Reducer subtractor, + final String storeName); /** - * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable} - * using default serializers and deserializers. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Combine the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) + * mapped} to the same key into a new instance of {@link KTable}. + * Records with {@code null} key are ignored. + * Combining implies that the type of the aggregate result is the same as the type of the input value + * (c.f. {@link #aggregate(Initializer, Aggregator, Aggregator, Serde, String)}). + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. + * The specified {@link Reducer adder} is applied for each update record and computes a new aggregate using the + * current aggregate and the record's value by adding the new record to the aggregate. + * The specified {@link Reducer substractor} is applied for each "replaced" record of the original {@link KTable} + * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" + * record from the aggregate. + * If there is no current aggregate the {@link Reducer} is not applied and the new aggregate will be the record's + * value as-is. + * Thus, {@code reduce(Reducer, Reducer, String)} can be used to compute aggregate functions like sum. + * For sum, the adder and substractor would work as follows: + *

      {@code
      +     * public class SumAdder implements Reducer {
      +     *   public Integer apply(Integer currentAgg, Integer newValue) {
      +     *     return currentAgg + newValue;
      +     *   }
      +     * }
            *
      -     * @param initializer   the instance of {@link Initializer}
      -     * @param adder         the instance of {@link Aggregator} for addition
      -     * @param subtractor   the instance of {@link Aggregator} for subtraction
      -     * @param storeName     the name of the underlying {@link KTable} state store
      -     * @param            the value type of the aggregated {@link KTable}
      -     * @return a {@link KTable} with same key and aggregated value type {@code T},
      -     *         containing aggregated values for each key
      +     * public class SumSubtractor implements Reducer {
      +     *   public Integer apply(Integer currentAgg, Integer oldValue) {
      +     *     return currentAgg - oldValue;
      +     *   }
      +     * }
      +     * }
      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * String storeName = storeSupplier.name();
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + * + * @param adder a {@link Reducer} that adds a new value to the aggregate result + * @param subtractor a {@link Reducer} that removed an old value from the aggregate result + * @param storeSupplier user defined state store supplier + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ - KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator subtractor, - String storeName); + KTable reduce(final Reducer adder, + final Reducer subtractor, + final StateStoreSupplier storeSupplier); /** - * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable} - * using default serializers and deserializers. - * The resulting {@link KTable} will be materialized in a state - * store provided by the {@link StateStoreSupplier}. + * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) + * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers. + * Records with {@code null} key are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it, + * for example, allows the result to have a different type than the input values. + * If the result value type does not match the {@link StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value + * serde} you should use {@link KGroupedTable#aggregate(Initializer, Aggregator, Aggregator, Serde, String) + * aggregate(Initializer, Aggregator, Aggregator, Serde, String)}. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * The specified {@link Initializer} is applied once directly before the first input record is processed to + * provide an initial intermediate aggregation result that is used to process the first record. + * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. + * The specified {@link Aggregator adder} is applied for each update record and computes a new aggregate using the + * current aggregate (or for the very first record using the intermediate aggregation result provided via the + * {@link Initializer}) and the record's value by adding the new record to the aggregate. + * The specified {@link Aggregator substractor} is applied for each "replaced" record of the original {@link KTable} + * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" + * record from the aggregate. + * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions + * like sum. + * For sum, the initializer, adder, and substractor would work as follows: + *

      {@code
      +     * // in this example, LongSerde.class must be set as default value serde in StreamsConfig
      +     * public class SumInitializer implements Initializer {
      +     *   public Long apply() {
      +     *     return 0L;
      +     *   }
      +     * }
      +     *
      +     * public class SumAdder implements Aggregator {
      +     *   public Long apply(String key, Integer newValue, Long aggregate) {
      +     *     return aggregate + newValue;
      +     *   }
      +     * }
      +     *
      +     * public class SumSubstractor implements Aggregator {
      +     *   public Long apply(String key, Integer oldValue, Long aggregate) {
      +     *     return aggregate - oldValue;
      +     *   }
      +     * }
      +     * }
      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param initializer the instance of {@link Initializer} - * @param adder the instance of {@link Aggregator} for addition - * @param subtractor the instance of {@link Aggregator} for subtraction - * @param storeSupplier user defined state store supplier {@link StateStoreSupplier} - * @param the value type of the aggregated {@link KTable} - * @return a {@link KTable} with same key and aggregated value type {@code T}, - * containing aggregated values for each key + * @param initializer a {@link Initializer} that provides an initial aggregate result value + * @param adder a {@link Aggregator} that adds a new record to the aggregate result + * @param subtractor a {@link Aggregator} that removed an old record from the aggregate result + * @param storeName the name of the underlying {@link KTable} state store + * @param the value type of the aggregated {@link KTable} + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ - KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator subtractor, - final StateStoreSupplier storeSupplier); + KTable aggregate(final Initializer initializer, + final Aggregator adder, + final Aggregator subtractor, + final String storeName); /** - * Count number of records of this stream by the selected key into a new instance of {@link KTable}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) + * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers. + * Records with {@code null} key are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it, + * for example, allows the result to have a different type than the input values. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * that can be queried using the provided {@code storeName}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * The specified {@link Initializer} is applied once directly before the first input record is processed to + * provide an initial intermediate aggregation result that is used to process the first record. + * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. + * The specified {@link Aggregator adder} is applied for each update record and computes a new aggregate using the + * current aggregate (or for the very first record using the intermediate aggregation result provided via the + * {@link Initializer}) and the record's value by adding the new record to the aggregate. + * The specified {@link Aggregator substractor} is applied for each "replaced" record of the original {@link KTable} + * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" + * record from the aggregate. + * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions + * like sum. + * For sum, the initializer, adder, and substractor would work as follows: + *

      {@code
      +     * public class SumInitializer implements Initializer {
      +     *   public Long apply() {
      +     *     return 0L;
      +     *   }
      +     * }
      +     *
      +     * public class SumAdder implements Aggregator {
      +     *   public Long apply(String key, Integer newValue, Long aggregate) {
      +     *     return aggregate + newValue;
      +     *   }
      +     * }
      +     *
      +     * public class SumSubstractor implements Aggregator {
      +     *   public Long apply(String key, Integer oldValue, Long aggregate) {
      +     *     return aggregate - oldValue;
      +     *   }
      +     * }
      +     * }
      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * + * @param initializer a {@link Initializer} that provides an initial aggregate result value + * @param adder a {@link Aggregator} that adds a new record to the aggregate result + * @param subtractor a {@link Aggregator} that removed an old record from the aggregate result + * @param aggValueSerde aggregate value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used * @param storeName the name of the underlying {@link KTable} state store - * @return a {@link KTable} with same key and {@link Long} value type as this {@link KGroupedTable}, - * containing the number of values for each key + * @param the value type of the aggregated {@link KTable} + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ - KTable count(String storeName); + KTable aggregate(final Initializer initializer, + final Aggregator adder, + final Aggregator subtractor, + final Serde aggValueSerde, + final String storeName); /** - * Count number of records of this stream by the selected key into a new instance of {@link KTable}. - * The resulting {@link KTable} will be materialized in a state - * store provided by the {@link StateStoreSupplier}. + * Aggregate the value of records of the original {@link KTable} that got {@link KTable#groupBy(KeyValueMapper) + * mapped} to the same key into a new instance of {@link KTable} using default serializers and deserializers. + * Records with {@code null} key are ignored. + * Aggregating is a generalization of {@link #reduce(Reducer, Reducer, String) combining via reduce(...)} as it, + * for example, allows the result to have a different type than the input values. + * The result is written into a local {@link KeyValueStore} (which is basically an ever-updating materialized view) + * provided by the given {@code storeSupplier}. + * Furthermore, updates to the store are sent downstream into a {@link KTable} changelog stream. + *

      + * The specified {@link Initializer} is applied once directly before the first input record is processed to + * provide an initial intermediate aggregation result that is used to process the first record. + * Each update to the original {@link KTable} results in a two step update of the result {@link KTable}. + * The specified {@link Aggregator adder} is applied for each update record and computes a new aggregate using the + * current aggregate (or for the very first record using the intermediate aggregation result provided via the + * {@link Initializer}) and the record's value by adding the new record to the aggregate. + * The specified {@link Aggregator substractor} is applied for each "replaced" record of the original {@link KTable} + * and computes a new aggregate using the current aggregate and the record's value by "removing" the "replaced" + * record from the aggregate. + * Thus, {@code aggregate(Initializer, Aggregator, Aggregator, String)} can be used to compute aggregate functions + * like sum. + * For sum, the initializer, adder, and substractor would work as follows: + *

      {@code
      +     * public class SumInitializer implements Initializer {
      +     *   public Long apply() {
      +     *     return 0L;
      +     *   }
      +     * }
      +     *
      +     * public class SumAdder implements Aggregator {
      +     *   public Long apply(String key, Integer newValue, Long aggregate) {
      +     *     return aggregate + newValue;
      +     *   }
      +     * }
            *
      -     * @param storeSupplier user defined state store supplier {@link StateStoreSupplier}
      -     * @return a {@link KTable} with same key and {@link Long} value type as this {@link KGroupedTable},
      -     * containing the number of values for each key
      +     * public class SumSubstractor implements Aggregator {
      +     *   public Long apply(String key, Integer oldValue, Long aggregate) {
      +     *     return aggregate - oldValue;
      +     *   }
      +     * }
      +     * }
      + * Not all updates might get sent downstream, as an internal cache is used to deduplicate consecutive updates to + * the same key. + * The rate of propagated updates depends on your input data rate, the number of distinct keys, the number of + * parallel running Kafka Streams instances, and the {@link StreamsConfig configuration} parameters for + * {@link StreamsConfig#CACHE_MAX_BYTES_BUFFERING_CONFIG cache size}, and + * {@link StreamsConfig#COMMIT_INTERVAL_MS_CONFIG commit intervall}. + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ... // counting words
      +     * String storeName = storeSupplier.name();
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-word";
      +     * Long countForWord = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. + *

      + * For failure and recovery the store will be backed by an internal changelog topic that will be created in Kafka. + * The changelog topic will be named "${applicationId}-${storeName}-changelog", where "applicationId" is + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is the + * provide {@code storeName}, and "-changelog" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + * + * @param initializer a {@link Initializer} that provides an initial aggregate result value + * @param adder a {@link Aggregator} that adds a new record to the aggregate result + * @param subtractor a {@link Aggregator} that removed an old record from the aggregate result + * @param storeSupplier user defined state store supplier + * @param the value type of the aggregated {@link KTable} + * @return a {@link KTable} that contains "update" records with unmodified keys, and values that represent the + * latest (rolling) aggregate for each key */ - KTable count(final StateStoreSupplier storeSupplier); + KTable aggregate(final Initializer initializer, + final Aggregator adder, + final Aggregator subtractor, + final StateStoreSupplier storeSupplier); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 8bc54e2256ab7..1b49f40ababb2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -16,27 +16,33 @@ */ package org.apache.kafka.streams.kstream; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.internals.WindowedSerializer; +import org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; +import org.apache.kafka.streams.processor.TopologyBuilder; /** - * {@link KStream} is an abstraction of a record stream of key-value pairs, - * i.e., each record is an independent entity/event in the real world. + * {@code KStream} is an abstraction of a record stream of {@link KeyValue} pairs, i.e., each record is an + * independent entity/event in the real world. * For example a user X might buy two items I1 and I2, and thus there might be two records {@code , } * in the stream. *

      - * A {@link KStream} is either defined from one or multiple Kafka topics that are consumed message by message or - * the result of a {@link KStream} transformation. - * A {@link KTable} can also be converted into a {@link KStream}. + * A {@code KStream} is either {@link KStreamBuilder#stream(String...) defined from one or multiple Kafka topics} that + * are consumed message by message or the result of a {@code KStream} transformation. + * A {@link KTable} can also be {@link KTable#toStream() converted} into a {@code KStream}. *

      - * A {@link KStream} can be transformed record by record, joined with another {@link KStream} or {@link KTable}, or - * can be aggregated into a {@link KTable}. - * Kafka Streams DSL can be mixed-and-matched with Processor API (PAPI) (c.f. - * {@link org.apache.kafka.streams.processor.TopologyBuilder TopologyBuilder}) via + * A {@code KStream} can be transformed record by record, joined with another {@code KStream}, {@link KTable}, + * {@link GlobalKTable}, or can be aggregated into a {@link KTable}. + * Kafka Streams DSL can be mixed-and-matched with Processor API (PAPI) (c.f. {@link TopologyBuilder}) via * {@link #process(ProcessorSupplier, String...) process(...)}, * {@link #transform(TransformerSupplier, String...) transform(...)}, and * {@link #transformValues(ValueTransformerSupplier, String...) transformValues(...)}. @@ -44,27 +50,32 @@ * @param Type of keys * @param Type of values * @see KTable + * @see KGroupedStream + * @see KStreamBuilder#stream(String...) */ @SuppressWarnings("unused") @InterfaceStability.Unstable public interface KStream { /** - * Create a new {@link KStream} that consists of all records of this stream which satisfy a predicate. - * All records that do not satisfy the predicate are dropped. This is a stateless record-by-record operation. + * Create a new {@code KStream} that consists of all records of this stream which satisfy the given predicate. + * All records that do not satisfy the predicate are dropped. + * This is a stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record - * @return a {@link KStream} that contains only those records that satisfy the given predicate + * @return a {@code KStream} that contains only those records that satisfy the given predicate * @see #filterNot(Predicate) */ KStream filter(Predicate predicate); /** - * Create a new {@link KStream} that consists all records of this stream which do not satisfy a predicate. - * All records that do satisfy the predicate are dropped. This is a stateless record-by-record operation. + * Create a new {@code KStream} that consists all records of this stream which do not satisfy the given + * predicate. + * All records that do satisfy the predicate are dropped. + * This is a stateless record-by-record operation. * * @param predicate a filter {@link Predicate} that is applied to each record - * @return a {@link KStream} that contains only those records that do not satisfy the given predicate + * @return a {@code KStream} that contains only those records that do not satisfy the given predicate * @see #filter(Predicate) */ KStream filterNot(Predicate predicate); @@ -79,7 +90,7 @@ public interface KStream { * extracting a key from the value within your {@link KeyValueMapper}. The example below computes the new key as the * length of the value string. *

      {@code
      -     * KStream keyLessStream = builder.from("key-less-topic");
      +     * KStream keyLessStream = builder.stream("key-less-topic");
            * KStream keyedStream = keyLessStream.selectKey(new KeyValueMapper {
            *     Integer apply(Byte[] key, String value) {
            *         return value.length();
      @@ -88,11 +99,11 @@ public interface KStream {
            * }
      *

      * Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@link KStream}. + * join) is applied to the result {@code KStream}. * * @param mapper a {@link KeyValueMapper} that computes a new key for each record * @param the new key type of the result stream - * @return a {@link KStream} that contains records with new key (possibly of different type) and unmodified value + * @return a {@code KStream} that contains records with new key (possibly of different type) and unmodified value * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) @@ -101,15 +112,16 @@ public interface KStream { KStream selectKey(KeyValueMapper mapper); /** - * Transform each record of the input stream into a new record in the output stream - * (both key and value type can be altered arbitrarily). + * Transform each record of the input stream into a new record in the output stream (both key and value type can be + * altered arbitrarily). * The provided {@link KeyValueMapper} is applied to each input record and computes a new output record. * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation. + * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for + * stateful record transformation). *

      * The example below normalizes the String key to upper-case letters and counts the number of token of the value string. *

      {@code
      -     * KStream inputStream = builder.from("topic");
      +     * KStream inputStream = builder.stream("topic");
            * KStream outputStream = inputStream.map(new KeyValueMapper> {
            *     KeyValue apply(String key, String value) {
            *         return new KeyValue<>(key.toUpperCase(), value.split(" ").length);
      @@ -117,19 +129,21 @@ public interface KStream {
            * });
            * }
      *

      - * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and the return value must not be {@code null}. + * The provided {@link KeyValueMapper} must return a {@link KeyValue} type and must not return {@code null}. *

      * Mapping records might result in an internal data redistribution if a key based operator (like an aggregation or - * join) is applied to the result {@link KStream}. (cf. {@link #mapValues(ValueMapper)}) + * join) is applied to the result {@code KStream}. (cf. {@link #mapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes a new output record * @param the key type of the result stream * @param the value type of the result stream - * @return a {@link KStream} that contains records with new key and value (possibly both of different type) + * @return a {@code KStream} that contains records with new key and value (possibly both of different type) * @see #selectKey(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) * @see #flatMapValues(ValueMapper) + * @see #transform(TransformerSupplier, String...) + * @see #transformValues(ValueTransformerSupplier, String...) */ KStream map(KeyValueMapper> mapper); @@ -137,12 +151,13 @@ public interface KStream { * Transform the value of each input record into a new value (with possible new type) of the output record. * The provided {@link ValueMapper} is applied to each input record value and computes a new value for it. * Thus, an input record {@code } can be transformed into an output record {@code }. - * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)}). + * This is a stateless record-by-record operation (cf. + * {@link #transformValues(ValueTransformerSupplier, String...)} for stateful value transformation). *

      * The example below counts the number of token of the value string. *

      {@code
      -     * KStream inputStream = builder.from("topic");
      -     * KStream outputStream = inputStream.map(new ValueMapper {
      +     * KStream inputStream = builder.stream("topic");
      +     * KStream outputStream = inputStream.mapValues(new ValueMapper {
            *     Integer apply(String value) {
            *         return value.split(" ").length;
            *     }
      @@ -151,15 +166,16 @@ public interface KStream {
            * 

      * Setting a new value preserves data co-location with respect to the key. * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@link KStream}. (cf. {@link #map(KeyValueMapper)}) + * is applied to the result {@code KStream}. (cf. {@link #map(KeyValueMapper)}) * * @param mapper a {@link ValueMapper} that computes a new output value * @param the value type of the result stream - * @return a {@link KStream} that contains records with unmodified key and new values (possibly of different type) + * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #flatMapValues(ValueMapper) + * @see #transform(TransformerSupplier, String...) * @see #transformValues(ValueTransformerSupplier, String...) */ KStream mapValues(ValueMapper mapper); @@ -169,12 +185,13 @@ public interface KStream { * can be altered arbitrarily). * The provided {@link KeyValueMapper} is applied to each input record and computes zero or more output records. * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)}). + * This is a stateless record-by-record operation (cf. {@link #transform(TransformerSupplier, String...)} for + * stateful record transformation). *

      * The example below splits input records {@code } containing sentences as values into their words * and emit a record {@code } for each word. *

      {@code
      -     * KStream inputStream = builder.from("topic");
      +     * KStream inputStream = builder.stream("topic");
            * KStream outputStream = inputStream.flatMap(new KeyValueMapper>> {
            *     Iterable> apply(byte[] key, String value) {
            *         String[] tokens = value.split(" ");
      @@ -193,32 +210,34 @@ public interface KStream {
            * and the return value must not be {@code null}.
            * 

      * Flat-mapping records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@link KStream}. (cf. {@link #flatMapValues(ValueMapper)}) + * or join) is applied to the result {@code KStream}. (cf. {@link #flatMapValues(ValueMapper)}) * * @param mapper a {@link KeyValueMapper} that computes the new output records * @param the key type of the result stream * @param the value type of the result stream - * @return a {@link KStream} that contains more or less records with new key and value (possibly of different type) + * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #mapValues(ValueMapper) * @see #flatMapValues(ValueMapper) * @see #transform(TransformerSupplier, String...) + * @see #transformValues(ValueTransformerSupplier, String...) */ KStream flatMap(final KeyValueMapper>> mapper); /** - * Create a new instance of {@link KStream} by transforming the value of each element in this stream into zero or - * more values with the same key in the new stream. + * Create a new {@code KStream} by transforming the value of each record in this stream into zero or more values + * with the same key in the new stream. * Transform the value of each input record into zero or more records with the same (unmodified) key in the output * stream (value type can be altered arbitrarily). * The provided {@link ValueMapper} is applied to each input record and computes zero or more output values. * Thus, an input record {@code } can be transformed into output records {@code , , ...}. - * This is a stateless record-by-record operation. + * This is a stateless record-by-record operation (cf. {@link #transformValues(ValueTransformerSupplier, String...)} + * for stateful value transformation). *

      * The example below splits input records {@code } containing sentences as values into their words. *

      {@code
      -     * KStream inputStream = builder.from("topic");
      +     * KStream inputStream = builder.stream("topic");
            * KStream outputStream = inputStream.flatMap(new ValueMapper> {
            *     Iterable apply(String value) {
            *         return Arrays.asList(value.split(" "));
      @@ -231,22 +250,27 @@ public interface KStream {
            * 

      * Splitting a record into multiple records with the same key preserves data co-location with respect to the key. * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join) - * is applied to the result {@link KStream}. (cf. {@link #flatMap(KeyValueMapper)}) + * is applied to the result {@code KStream}. (cf. {@link #flatMap(KeyValueMapper)}) * * @param processor a {@link ValueMapper} the computes the new output values * @param the value type of the result stream - * @return a {@link KStream} that contains more or less records with unmodified keys and new values of different type + * @return a {@code KStream} that contains more or less records with unmodified keys and new values of different type * @see #selectKey(KeyValueMapper) * @see #map(KeyValueMapper) * @see #flatMap(KeyValueMapper) * @see #mapValues(ValueMapper) + * @see #transform(TransformerSupplier, String...) + * @see #transformValues(ValueTransformerSupplier, String...) */ KStream flatMapValues(final ValueMapper> processor); /** - * Print the elements of this stream to {@code System.out}. - * This function will use the generated name of the parent processor node to label the key/value pairs printed out - * to the console. + * Print the records of this stream to {@code System.out}. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the console. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, * {@link Integer} etc. to get meaningful information. @@ -254,22 +278,25 @@ public interface KStream { void print(); /** - * Print the elements of this stream to {@code System.out}. - * This function will use the given name to label the key/value pairs printed out to the console. + * Print the records of this stream to {@code System.out}. + * This function will use the given name to label the key/value pairs printed to the console. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, * {@link Integer} etc. to get meaningful information. * - * @param streamName the name used to label the key/value pairs printed out to the console + * @param streamName the name used to label the key/value pairs printed to the console */ void print(final String streamName); /** - * Print the elements of this stream to {@code System.out}. - * This function will use the generated name of the parent processor node to label the key/value pairs printed out - * to the console. + * Print the records of this stream to {@code System.out}. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the console. *

      - * The provided serde will be use to deserialize the key or value in case the type is {@code byte[]} before calling + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, @@ -282,9 +309,9 @@ void print(final Serde keySerde, final Serde valSerde); /** - * Print the elements of this stream to {@code System.out}. + * Print the records of this stream to {@code System.out}. *

      - * The provided serde will be use to deserialize the key or value in case the type is {@code byte[]} before calling + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, @@ -292,16 +319,19 @@ void print(final Serde keySerde, * * @param keySerde key serde used to deserialize key if type is {@code byte[]}, * @param valSerde value serde used to deserialize value if type is {@code byte[]}, - * @param streamName the name used to label the key/value pairs printed out to the console + * @param streamName the name used to label the key/value pairs printed to the console */ void print(final Serde keySerde, final Serde valSerde, final String streamName); /** - * Write the elements of this stream to a file at the given path. - * This function will use the generated name of the parent processor node to label the key/value pairs printed out - * to the file. + * Write the records of this stream to a file at the given path. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the file. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, * {@link Integer} etc. to get meaningful information. @@ -311,8 +341,11 @@ void print(final Serde keySerde, void writeAsText(final String filePath); /** - * Write the elements of this stream to a file at the given path. - * This function will use the given name to label the key/value printed out to the file. + * Write the records of this stream to a file at the given path. + * This function will use the given name to label the key/value printed to the file. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. *

      * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, * {@link Integer} etc. to get meaningful information. @@ -324,9 +357,9 @@ void writeAsText(final String filePath, final String streamName); /** - * Write the elements of this stream to a file at the given path. - * This function will use the generated name of the parent processor node to label the key/value pairs printed out - * to the file. + * Write the records of this stream to a file at the given path. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the file. *

      * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling * {@code toString()} on the deserialized object. @@ -343,8 +376,8 @@ void writeAsText(final String filePath, final Serde valSerde); /** - * Write the elements of this stream to a file at the given path. - * This function will use the given name to label the key/value printed out to the file. + * Write the records of this stream to a file at the given path. + * This function will use the given name to label the key/value printed to the file. *

      * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} * before calling {@code toString()} on the deserialized object. @@ -363,7 +396,7 @@ void writeAsText(final String filePath, final Serde valSerde); /** - * Perform an action on each record of {@link KStream}. + * Perform an action on each record of {@code KStream}. * This is a stateless record-by-record operation (cf. {@link #process(ProcessorSupplier, String...)}). * Note that this is a terminal operation that returns void. * @@ -373,90 +406,82 @@ void writeAsText(final String filePath, void foreach(final ForeachAction action); /** - * Creates an array of {@link KStream} from this stream by branching the elements in the original stream based on + * Creates an array of {@code KStream} from this stream by branching the records in the original stream based on * the supplied predicates. - * Each element is evaluated against the supplied predicates, and predicates are evaluated in order. + * Each record is evaluated against the supplied predicates, and predicates are evaluated in order. * Each stream in the result array corresponds position-wise (index) to the predicate in the supplied predicates. - * The branching happens on first-match: An element in the original stream is assigned to the corresponding result + * The branching happens on first-match: A record in the original stream is assigned to the corresponding result * stream for the first predicate that evaluates to true, and is assigned to this stream only. - * An element will be dropped if none of the predicates evaluate to true. + * A record will be dropped if none of the predicates evaluate to true. * This is a stateless record-by-record operation. * * @param predicates the ordered list of {@link Predicate} instances - * @return multiple distinct substreams of this {@link KStream} + * @return multiple distinct substreams of this {@code KStream} */ @SuppressWarnings("unchecked") KStream[] branch(final Predicate... predicates); /** - * Materialize this stream to a topic and creates a new instance of {@link KStream} from the topic using default - * serializers and deserializers and producer's - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner}. + * Materialize this stream to a topic and creates a new {@code KStream} from the topic using default serializers and + * deserializers and producer's {@link DefaultPartitioner}. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). *

      - * This is equivalent to calling {@link #to(String) #to(someTopicName)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...) KStreamBuilder#stream(someTopicName)}. + * This is equivalent to calling {@link #to(String) #to(someTopicName)} and {@link KStreamBuilder#stream(String...) + * KStreamBuilder#stream(someTopicName)}. * * @param topic the topic name - * @return a {@link KStream} that contains the exact same records as this {@link KStream} + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} */ KStream through(final String topic); /** - * Materialize this stream to a topic and creates a new instance of {@link KStream} from the topic using default - * serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of - * records to partitions. + * Materialize this stream to a topic and creates a new {@code KStream} from the topic using default serializers and + * deserializers and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). *

      * This is equivalent to calling {@link #to(StreamPartitioner, String) #to(StreamPartitioner, someTopicName)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...) KStreamBuilder#stream(someTopicName)}. + * {@link KStreamBuilder#stream(String...) KStreamBuilder#stream(someTopicName)}. * * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified producer's - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner} - * will be used + * if not specified producer's {@link DefaultPartitioner} will be used * @param topic the topic name - * @return a {@link KStream} that contains the exact same records as this {@link KStream} + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} */ KStream through(final StreamPartitioner partitioner, final String topic); /** - * Materialize this stream to a topic, and creates a new instance of {@link KStream} from the topic. + * Materialize this stream to a topic, and creates a new {@code KStream} from the topic. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). *

      - * If {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer - * WindowedSerializer} for the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner - * WindowedStreamPartitioner} is used—otherwise producer's - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner} is used. + * If {@code keySerde} provides a {@link WindowedSerializer} for the key {@link WindowedStreamPartitioner} is + * used—otherwise producer's {@link DefaultPartitioner} is used. *

      * This is equivalent to calling {@link #to(Serde, Serde, String) #to(keySerde, valSerde, someTopicName)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(Serde, Serde, String...) - * KStreamBuilder#stream(keySerde, valSerde, someTopicName)}. + * {@link KStreamBuilder#stream(Serde, Serde, String...) KStreamBuilder#stream(keySerde, valSerde, someTopicName)}. * * @param keySerde key serde used to send key-value pairs, * if not specified the default key serde defined in the configuration will be used * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name - * @return a {@link KStream} that contains the exact same records as this {@link KStream} + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} */ KStream through(final Serde keySerde, final Serde valSerde, final String topic); /** - * Materialize this stream to a topic and creates a new instance of {@link KStream} from the topic - * using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * Materialize this stream to a topic and creates a new {@code KStream} from the topic using a customizable + * {@link StreamPartitioner} to determine the distribution of records to partitions. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). *

      * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String) #to(keySerde, valSerde, - * StreamPartitioner, someTopicName)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(Serde, Serde, String...) + * StreamPartitioner, someTopicName)} and {@link KStreamBuilder#stream(Serde, Serde, String...) * KStreamBuilder#stream(keySerde, valSerde, someTopicName)}. * * @param keySerde key serde used to send key-value pairs, @@ -464,14 +489,11 @@ KStream through(final Serde keySerde, * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified and {@code keySerde} provides a - * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer WindowedSerializer} for - * the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner - * WindowedStreamPartitioner} will be used—otherwise - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner} will + * if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key + * {@link WindowedStreamPartitioner} will be used—otherwise {@link DefaultPartitioner} will * be used * @param topic the topic name - * @return a {@link KStream} that contains the exact same records as this {@link KStream} + * @return a {@code KStream} that contains the exact same (and potentially repartitioned) records as this {@code KStream} */ KStream through(final Serde keySerde, final Serde valSerde, @@ -480,7 +502,7 @@ KStream through(final Serde keySerde, /** * Materialize this stream to a topic using default serializers specified in the config and producer's - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner}. + * {@link DefaultPartitioner}. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). * @@ -495,20 +517,16 @@ KStream through(final Serde keySerde, * started). * * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified producer's - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner} will - * be used + * if not specified producer's {@link DefaultPartitioner} will be used * @param topic the topic name */ void to(final StreamPartitioner partitioner, final String topic); /** - * Materialize this stream to a topic. If {@code keySerde} provides a - * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer WindowedSerializer} for the key - * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner WindowedStreamPartitioner} is - * used—otherwise producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner - * DefaultPartitioner} is used. + * Materialize this stream to a topic. If {@code keySerde} provides a {@link WindowedSerializer WindowedSerializer} + * for the key {@link WindowedStreamPartitioner} is used—otherwise producer's {@link DefaultPartitioner} is + * used. * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is * started). * @@ -533,11 +551,8 @@ void to(final Serde keySerde, * @param valSerde value serde used to send key-value pairs, * if not specified the default serde defined in the configs will be used * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified and {@code keySerde} provides a - * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer WindowedSerializer} for - * the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner - * WindowedStreamPartitioner} will be used—otherwise - * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner DefaultPartitioner} will + * if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key + * {@link WindowedStreamPartitioner} will be used—otherwise {@link DefaultPartitioner} will * be used * @param topic the topic name */ @@ -572,7 +587,7 @@ void to(final Serde keySerde, * }

      *

      * Within the {@link Transformer}, the state is obtained via the - * {@link org.apache.kafka.streams.processor.ProcessorContext ProcessorContext}. + * {@link ProcessorContext}. * To trigger periodic actions via {@link Transformer#punctuate(long) punctuate()}, a schedule must be registered. * The {@link Transformer} must return a {@link KeyValue} type in {@link Transformer#transform(Object, Object) * transform()} and {@link Transformer#punctuate(long) punctuate()}. @@ -611,19 +626,19 @@ void to(final Serde keySerde, * }

      *

      * Transforming records might result in an internal data redistribution if a key based operator (like an aggregation - * or join) is applied to the result {@link KStream}. + * or join) is applied to the result {@code KStream}. * (cf. {@link #transformValues(ValueTransformerSupplier, String...)}) * * @param transformerSupplier a instance of {@link TransformerSupplier} that generates a {@link Transformer} * @param stateStoreNames the names of the state stores used by the processor * @param the key type of the new stream * @param the value type of the new stream - * @return a {@link KStream} that contains more or less records with new key and value (possibly of different type) + * @return a {@code KStream} that contains more or less records with new key and value (possibly of different type) * @see #flatMap(KeyValueMapper) * @see #transformValues(ValueTransformerSupplier, String...) * @see #process(ProcessorSupplier, String...) */ - KStream transform(final TransformerSupplier> transformerSupplier, + KStream transform(final TransformerSupplier> transformerSupplier, final String... stateStoreNames); /** @@ -651,11 +666,11 @@ KStream transform(final TransformerSupplier *

      * Within the {@link ValueTransformer}, the state is obtained via the - * {@link org.apache.kafka.streams.processor.ProcessorContext ProcessorContext}. + * {@link ProcessorContext}. * To trigger periodic actions via {@link ValueTransformer#punctuate(long) punctuate()}, a schedule must be * registered. * In contrast to {@link #transform(TransformerSupplier, String...) transform()}, no additional {@link KeyValue} - * pairs should be emitted via {@link org.apache.kafka.streams.processor.ProcessorContext#forward(Object, Object) + * pairs should be emitted via {@link ProcessorContext#forward(Object, Object) * ProcessorContext.forward()}. *

      {@code
            * new ValueTransformerSupplier() {
      @@ -688,13 +703,13 @@  KStream transform(final TransformerSupplier
            * Setting a new value preserves data co-location with respect to the key.
            * Thus, no internal data redistribution is required if a key based operator (like an aggregation or join)
      -     * is applied to the result {@link KStream}. (cf. {@link #transform(TransformerSupplier, String...)})
      +     * is applied to the result {@code KStream}. (cf. {@link #transform(TransformerSupplier, String...)})
            *
            * @param valueTransformerSupplier a instance of {@link ValueTransformerSupplier} that generates a
            *                                 {@link ValueTransformer}
            * @param stateStoreNames          the names of the state stores used by the processor
            * @param                      the value type of the result stream
      -     * @return a {@link KStream} that contains records with unmodified key and new values (possibly of different type)
      +     * @return a {@code KStream} that contains records with unmodified key and new values (possibly of different type)
            * @see #mapValues(ValueMapper)
            * @see #transform(TransformerSupplier, String...)
            */
      @@ -702,11 +717,11 @@  KStream transformValues(final ValueTransformerSupplier
            * In order to assign a state, the state must be created and registered beforehand:
      @@ -724,9 +739,9 @@  KStream transformValues(final ValueTransformerSupplier
            * 

      - * Within the {@link org.apache.kafka.streams.processor.Processor Processor}, the state is obtained via the - * {@link org.apache.kafka.streams.processor.ProcessorContext ProcessorContext}. - * To trigger periodic actions via {@link org.apache.kafka.streams.processor.Processor#punctuate(long) punctuate()}, + * Within the {@link Processor}, the state is obtained via the + * {@link ProcessorContext}. + * To trigger periodic actions via {@link Processor#punctuate(long) punctuate()}, * a schedule must be registered. *

      {@code
            * new ProcessorSupplier() {
      @@ -739,7 +754,7 @@  KStream transformValues(final ValueTransformerSupplier KStream transformValues(final ValueTransformerSupplier
            *
      -     * @param processorSupplier a instance of {@link ProcessorSupplier} that generates a
      -     *                          {@link org.apache.kafka.streams.processor.Processor}
      +     * @param processorSupplier a instance of {@link ProcessorSupplier} that generates a {@link Processor}
            * @param stateStoreNames   the names of the state store used by the processor
            * @see #foreach(ForeachAction)
            * @see #transform(TransformerSupplier, String...)
      @@ -765,39 +779,62 @@ void process(final ProcessorSupplier processorSupplier,
                        final String... stateStoreNames);
       
           /**
      -     * Group the records of this {@link KStream} on a new key that is selected using the provided {@link KeyValueMapper}.
      +     * Group the records by their current key into a {@link KGroupedStream} while preserving the original values
      +     * and default serializers and deserializers.
            * Grouping a stream on the record key is required before an aggregation operator can be applied to the data
            * (cf. {@link KGroupedStream}).
      -     * The {@link KeyValueMapper} selects a new key (with potentially different type) while preserving the original values.
      -     * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}.
      +     * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}.
            * 

      - * Because a new key is selected, an internal repartitioning topic will be created in Kafka. + * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, + * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)}, or + * {@link #transform(TransformerSupplier, String...)}), and no data redistribution happened afterwards (e.g., via + * {@link #through(String)}) an internal repartitioning topic will be created in Kafka. * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in - * {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is + * an internally generated name, and "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * All data of this stream will be redistributed through the repartitioning topic by writing all records to - * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. + * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all + * records to it, and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned + * correctly on its key. + * If the last key changing operator changed the key type, it is recommended to use + * {@link #groupByKey(Serde, Serde)} instead. + * + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @see #groupBy(KeyValueMapper) + */ + KGroupedStream groupByKey(); + + /** + * Group the records by their current key into a {@link KGroupedStream} while preserving the original values. + * Grouping a stream on the record key is required before an aggregation operator can be applied to the data + * (cf. {@link KGroupedStream}). + * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. *

      - * This is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey(Serde, Serde)}. + * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, + * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)}, or + * {@link #transform(TransformerSupplier, String...)}), and no data redistribution happened afterwards (e.g., via + * {@link #through(String)}) an internal repartitioning topic will be created in Kafka. + * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is + * an internally generated name, and "-repartition" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + *

      + * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all + * records to it, and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned + * correctly on its key. * - * @param selector a {@link KeyValueMapper} that computes a new key for grouping * @param keySerde key serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used * @param valSerde value serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used - * @param the key type of the result {@link KGroupedStream} - * @return a {@link KGroupedStream} that contains the grouped records of the original {@link KStream} - * @see #groupByKey() + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} */ - KGroupedStream groupBy(final KeyValueMapper selector, - final Serde keySerde, - final Serde valSerde); + KGroupedStream groupByKey(final Serde keySerde, + final Serde valSerde); /** - * Group the records of this {@link KStream} on a new key that is selected using the provided {@link KeyValueMapper} + * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper} * and default serializers and deserializers. * Grouping a stream on the record key is required before an aggregation operator can be applied to the data * (cf. {@link KGroupedStream}). @@ -806,12 +843,11 @@ KGroupedStream groupBy(final KeyValueMapper * Because a new key is selected, an internal repartitioning topic will be created in Kafka. * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in - * {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is + * an internally generated name, and "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * All data of this stream will be redistributed through the repartitioning topic by writing all records to + * All data of this stream will be redistributed through the repartitioning topic by writing all records to it, * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. *

      * This operation is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey()}. @@ -819,71 +855,46 @@ KGroupedStream groupBy(final KeyValueMapper the key type of the result {@link KGroupedStream} - * @return a {@link KGroupedStream} that contains the grouped records of the original {@link KStream} + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} */ KGroupedStream groupBy(final KeyValueMapper selector); /** - * Group the records by their current key into a {@link KGroupedStream} while preserving the original values - * and default serializers and deserializers. + * Group the records of this {@code KStream} on a new key that is selected using the provided {@link KeyValueMapper}. * Grouping a stream on the record key is required before an aggregation operator can be applied to the data * (cf. {@link KGroupedStream}). - * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. + * The {@link KeyValueMapper} selects a new key (with potentially different type) while preserving the original values. + * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. *

      - * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)}, or - * {@link #transform(TransformerSupplier, String...)}), and no data redistribution happened afterwards (e.g., via - * {@link #through(String)}) an internal repartitioning topic will be created in Kafka. + * Because a new key is selected, an internal repartitioning topic will be created in Kafka. * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in - * {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * {@link StreamsConfig StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned - * correctly on its key. - * If the last key changing operator changed the key type, it is recommended to use - * {@link #groupByKey(Serde, Serde)} instead. - * - * @return a {@link KGroupedStream} that contains the grouped records of the original {@link KStream} - * @see #groupBy(KeyValueMapper) - */ - KGroupedStream groupByKey(); - - /** - * Group the records by their current key into a {@link KGroupedStream} while preserving the original values. - * Grouping a stream on the record key is required before an aggregation operator can be applied to the data - * (cf. {@link KGroupedStream}). - * If a record key is {@code null} the record will not be included in the resulting {@link KGroupedStream}. - *

      - * If a key changing operator was used before this operation (e.g., {@link #selectKey(KeyValueMapper)}, - * {@link #map(KeyValueMapper)}, {@link #flatMap(KeyValueMapper)}, or - * {@link #transform(TransformerSupplier, String...)}), and no data redistribution happened afterwards (e.g., via - * {@link #through(String)}) an internal repartitioning topic will be created in Kafka. - * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in - * {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. - * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + * All data of this stream will be redistributed through the repartitioning topic by writing all records to it, + * and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned on the new key. *

      - * For this case, all data of this stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the resulting {@link KGroupedStream} is partitioned - * correctly on its key. + * This is equivalent to calling {@link #selectKey(KeyValueMapper)} followed by {@link #groupByKey(Serde, Serde)}. * + * @param selector a {@link KeyValueMapper} that computes a new key for grouping * @param keySerde key serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used * @param valSerde value serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used - * @return a {@link KGroupedStream} that contains the grouped records of the original {@link KStream} + * @param the key type of the result {@link KGroupedStream} + * @return a {@link KGroupedStream} that contains the grouped records of the original {@code KStream} + * @see #groupByKey() */ - KGroupedStream groupByKey(final Serde keySerde, - final Serde valSerde); + KGroupedStream groupBy(final KeyValueMapper selector, + final Serde keySerde, + final Serde valSerde); /** - * Join records of this stream with another {@link KStream}'s records using windowed inner equi join with default + * Join records of this stream with another {@code KStream}'s records using windowed inner equi join with default * serializers and deserializers. - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      @@ -891,7 +902,7 @@ KGroupedStream groupByKey(final Serde keySerde, * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -916,34 +927,38 @@ KGroupedStream groupByKey(final Serde keySerde, * * *
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s. + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an + * internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key and within the joining window intervals * @see #leftJoin(KStream, ValueJoiner, JoinWindows) * @see #outerJoin(KStream, ValueJoiner, JoinWindows) @@ -953,8 +968,8 @@ KStream join(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@link KStream}'s records using windowed inner equi join. - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * Join records of this stream with another {@code KStream}'s records using windowed inner equi join. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      @@ -962,7 +977,7 @@ KStream join(final KStream otherStream, * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -987,29 +1002,32 @@ KStream join(final KStream otherStream, * * *
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s. + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, + * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param keySerde key serdes for materializing both streams, @@ -1020,7 +1038,7 @@ KStream join(final KStream otherStream, * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key and within the joining window intervals * @see #leftJoin(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) * @see #outerJoin(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) @@ -1033,21 +1051,21 @@ KStream join(final KStream otherStream, final Serde otherValueSerde); /** - * Join records of this stream with another {@link KStream}'s records using windowed left equi join with default + * Join records of this stream with another {@code KStream}'s records using windowed left equi join with default * serializers and deserializers. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join}, all records from this stream will * produce at least one output record (cf. below). - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@link KStream} that does not satisfy the join predicate the provided + * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided * {@link ValueJoiner} will be called with a {@code null} value for the other stream. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -1072,36 +1090,39 @@ KStream join(final KStream otherStream, * * *
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, + * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@link KStream} and within the joining window intervals + * this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows) * @see #outerJoin(KStream, ValueJoiner, JoinWindows) */ @@ -1110,20 +1131,20 @@ KStream leftJoin(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@link KStream}'s records using windowed left equi join. + * Join records of this stream with another {@code KStream}'s records using windowed left equi join. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) inner-join}, all records from * this stream will produce at least one output record (cf. below). - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of this {@link KStream} that does not satisfy the join predicate the provided + * Furthermore, for each input record of this {@code KStream} that does not satisfy the join predicate the provided * {@link ValueJoiner} will be called with a {@code null} value for the other stream. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -1148,29 +1169,33 @@ KStream leftJoin(final KStream otherStream, * * *
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an + * internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param keySerde key serdes for materializing the other stream, @@ -1181,9 +1206,9 @@ KStream leftJoin(final KStream otherStream, * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * this {@link KStream} and within the joining window intervals + * this {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) * @see #outerJoin(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) */ @@ -1195,22 +1220,22 @@ KStream leftJoin(final KStream otherStream, final Serde otherValueSerde); /** - * Join records of this stream with another {@link KStream}'s records using windowed left equi join with default + * Join records of this stream with another {@code KStream}'s records using windowed left equi join with default * serializers and deserializers. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows) inner-join} or * {@link #leftJoin(KStream, ValueJoiner, JoinWindows) left-join}, all records from both streams will produce at * least one output record (cf. below). - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@link KStream}s that does not satisfy the join predicate the provided + * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided * {@link ValueJoiner} will be called with a {@code null} value for the this/other stream, respectively. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -1235,36 +1260,39 @@ KStream leftJoin(final KStream otherStream, * * *
      <K3:ValueJoiner(null,c)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, + * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@link KStream} and within the joining window intervals + * both {@code KStream} and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows) * @see #leftJoin(KStream, ValueJoiner, JoinWindows) */ @@ -1273,21 +1301,21 @@ KStream outerJoin(final KStream otherStream, final JoinWindows windows); /** - * Join records of this stream with another {@link KStream}'s records using windowed left equi join. + * Join records of this stream with another {@code KStream}'s records using windowed left equi join. * In contrast to {@link #join(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) inner-join} or * {@link #leftJoin(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) left-join}, all records from both * streams will produce at least one output record (cf. below). - * The join is a primary key join with join attribute {@code thisKStream.key == otherKStream.key}. + * The join is computed on the records' key with join attribute {@code thisKStream.key == otherKStream.key}. * Furthermore, two records are only joined if their timestamps are close to each other as defined by the given * {@link JoinWindows}, i.e., the window defines an additional join predicate on the record timestamps. *

      * For each pair of records meeting both join predicates the provided {@link ValueJoiner} will be called to compute * a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * Furthermore, for each input record of both {@link KStream}s that does not satisfy the join predicate the provided + * Furthermore, for each input record of both {@code KStream}s that does not satisfy the join predicate the provided * {@link ValueJoiner} will be called with a {@code null} value for this/other stream, respectively. * If an input record key or value is {@code null} the record will not be included in the join operation and thus no - * output record will be added to the resulting {@link KStream}. + * output record will be added to the resulting {@code KStream}. *

      * Example (assuming all input records belong to the correct windows): * @@ -1312,29 +1340,32 @@ KStream outerJoin(final KStream otherStream, * * *
      <K3:ValueJoiner(null,c)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} (for one input stream) before doing the + * join, using a pre-created topic with the "correct" number of partitions. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner). * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen for one or both of the joining {@link KStream}s + * Repartitioning can happen for one or both of the joining {@code KStream}s. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. *

      - * Both of the joining {@link KStream}s will be materialized in local state stores with auto-generated store names. + * Both of the joining {@code KStream}s will be materialized in local state stores with auto-generated store names. * For failure and recovery each store will be backed by an internal changelog topic that will be created in Kafka. * The changelog topic will be named "${applicationId}-storeName-changelog", where "applicationId" is user-specified - * in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "storeName" is an - * internal generated name, and "-changelog" is a fixed suffix. + * in {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, + * "storeName" is an internally generated name, and "-changelog" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. * - * @param otherStream the {@link KStream} to be joined with this stream + * @param otherStream the {@code KStream} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param windows the specification of the {@link JoinWindows} * @param keySerde key serdes for materializing both streams, @@ -1345,9 +1376,9 @@ KStream outerJoin(final KStream otherStream, * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of - * both {@link KStream} and within the joining window intervals + * both {@code KStream}s and within the joining window intervals * @see #join(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) * @see #leftJoin(KStream, ValueJoiner, JoinWindows, Serde, Serde, Serde) */ @@ -1362,17 +1393,17 @@ KStream outerJoin(final KStream otherStream, * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join with default * serializers and deserializers. * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current (ie, processing time) internal + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current (i.e., processing time) internal * {@link KTable} state. * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

      - * For each {@link KStream} record that finds an corresponding record in {@link KTable} the provided + * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

      * Example: * @@ -1401,27 +1432,33 @@ KStream outerJoin(final KStream otherStream, * * *
      <K1:ValueJoiner(C,b)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} for this {@code KStream} before doing + * the join, using a pre-created topic with the same number of partitions as the given {@link KTable}. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner); + * cf. {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner)}. * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen only for this {@link KStream}s. + * Repartitioning can happen only for this {@code KStream} but not for the provided {@link KTable}. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. * * @param table the {@link KTable} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param the value type of the table * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoiner) + * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream join(final KTable table, final ValueJoiner joiner); @@ -1429,17 +1466,17 @@ KStream join(final KTable table, /** * Join records of this stream with {@link KTable}'s records using non-windowed inner equi join. * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current (ie, processing time) internal + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current (i.e., processing time) internal * {@link KTable} state. * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

      - * For each {@link KStream} record that finds an corresponding record in {@link KTable} the provided + * For each {@code KStream} record that finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * The key of the result record is the same as for both joining input records. - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

      * Example: * @@ -1468,31 +1505,37 @@ KStream join(final KTable table, * * *
      <K1:ValueJoiner(C,b)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} for this {@code KStream} before doing + * the join, using a pre-created topic with the same number of partitions as the given {@link KTable}. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner); + * cf. {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner)}. * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen only for this {@link KStream}s. + * Repartitioning can happen only for this {@code KStream} but not for the provided {@link KTable}. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. * - * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param keySerde key serdes for materializing this stream. - * If not specified the default serdes defined in the configs will be used - * @param valSerde value serdes for materializing this stream, - * if not specified the default serdes defined in the configs will be used - * @param the value type of the table - * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given + * @param table the {@link KTable} to be joined with this stream + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param keySerde key serdes for materializing this stream. + * If not specified the default serdes defined in the configs will be used + * @param valSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the given * {@link ValueJoiner}, one for each matched record-pair with the same key * @see #leftJoin(KTable, ValueJoiner, Serde, Serde) + * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream join(final KTable table, final ValueJoiner joiner, @@ -1502,21 +1545,21 @@ KStream join(final KTable table, /** * Join records of this stream with {@link KTable}'s records using non-windowed left equi join with default * serializers and deserializers. - * In contrast to {@link #join(KTable, ValueJoiner)} inner-join}, all records from this stream will produce an + * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from this stream will produce an * output record (cf. below). * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current (ie, processing time) internal + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current (i.e., processing time) internal * {@link KTable} state. * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

      - * For each {@link KStream} record weather or not it finds an corresponding record in {@link KTable} the provided + * For each {@code KStream} record weather or not it finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. * The key of the result record is the same as for both joining input records. - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

      * Example: * @@ -1545,48 +1588,54 @@ KStream join(final KTable table, * * *
      <K1:ValueJoiner(C,b)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} for this {@code KStream} before doing + * the join, using a pre-created topic with the same number of partitions as the given {@link KTable}. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner); + * cf. {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner)}. * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen only for this {@link KStream}s. + * Repartitioning can happen only for this {@code KStream} but not for the provided {@link KTable}. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. * * @param table the {@link KTable} to be joined with this stream * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records * @param the value type of the table * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@link KStream} record + * @return a {@code KStream} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoiner) + * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream leftJoin(final KTable table, final ValueJoiner joiner); /** * Join records of this stream with {@link KTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(KTable, ValueJoiner)} inner-join}, all records from this stream will produce an + * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from this stream will produce an * output record (cf. below). * The join is a primary key table lookup join with join attribute {@code stream.key == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current (ie, processing time) internal + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current (i.e., processing time) internal * {@link KTable} state. * In contrast, processing {@link KTable} input records will only update the internal {@link KTable} state and * will not produce any result records. *

      - * For each {@link KStream} record weather or not it finds an corresponding record in {@link KTable} the provided + * For each {@code KStream} record weather or not it finds a corresponding record in {@link KTable} the provided * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. * If no {@link KTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. * The key of the result record is the same as for both joining input records. - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. *

      * Example: * @@ -1615,99 +1664,107 @@ KStream leftJoin(final KTable table, * * *
      <K1:ValueJoiner(C,b)>
      - * Both input streams need to be co-partitioned on the join key. + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * If this is not the case, you would need to call {@link #through(String)} for this {@code KStream} before doing + * the join, using a pre-created topic with the same number of partitions as the given {@link KTable}. + * Furthermore, both input streams need to be co-partitioned on the join key (i.e., use the same partitioner); + * cf. {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner)}. * If this requirement is not met, Kafka Streams will automatically repartition the data, i.e., it will create an * internal repartitioning topic in Kafka and write and re-read the data via this topic before the actual join. * The repartitioning topic will be named "${applicationId}-XXX-repartition", where "applicationId" is - * user-specified in {@link org.apache.kafka.streams.StreamsConfig StreamsConfig} via parameter - * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internal - * generated name, and "-repartition" is a fixed suffix. + * user-specified in {@link StreamsConfig} via parameter + * {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is an internally generated name, and + * "-repartition" is a fixed suffix. * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. *

      - * Repartitioning can happen only for this {@link KStream}s. + * Repartitioning can happen only for this {@code KStream} but not for the provided {@link KTable}. * For this case, all data of the stream will be redistributed through the repartitioning topic by writing all - * records to and rereading all records from it, such that the join input {@link KStream} is partitioned correctly - * on its key. + * records to it, and rereading all records from it, such that the join input {@code KStream} is partitioned + * correctly on its key. * - * @param table the {@link KTable} to be joined with this stream - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param keySerde key serdes for materializing this stream. - * If not specified the default serdes defined in the configs will be used - * @param valSerde value serdes for materializing this stream, - * if not specified the default serdes defined in the configs will be used - * @param the value type of the table - * @param the value type of the result stream - * @return a {@link KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@link KStream} record + * @param table the {@link KTable} to be joined with this stream + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param keySerde key serdes for materializing this stream. + * If not specified the default serdes defined in the configs will be used + * @param valSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the table + * @param the value type of the result stream + * @return a {@code KStream} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(KTable, ValueJoiner, Serde, Serde) + * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream leftJoin(final KTable table, final ValueJoiner joiner, final Serde keySerde, final Serde valSerde); + /** + * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join. + * The join is a primary key table lookup join with join attribute + * {@code keyValueMapper.map(stream.keyValue) == table.key}. + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} + * state. + * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} + * state and will not produce any result records. + *

      + * For each {@code KStream} record that finds a corresponding record in {@link GlobalKTable} the provided + * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. + * The key of the result record is the same as the key of this {@code KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. + * + * @param globalKTable the {@link GlobalKTable} to be joined with this stream + * @param keyValueMapper instance of {@link KeyValueMapper} used to map from the (key, value) of this stream + * to the key of the {@link GlobalKTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one output for each input {@code KStream} record + * @see #leftJoin(GlobalKTable, KeyValueMapper, ValueJoiner) + */ + KStream join(final GlobalKTable globalKTable, + final KeyValueMapper keyValueMapper, + final ValueJoiner joiner); /** * Join records of this stream with {@link GlobalKTable}'s records using non-windowed left equi join. - * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner)} inner-join}, all records from this stream will produce an - * output record (cf. below). - * The join is a primary key table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal - * {@link GlobalKTable} state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} state and - * will not produce any result records. - *

      - * For each {@link KStream} record whether or not it finds an corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to {@link ValueJoiner}. - * The key of the result record is the same as this {@link KStream} - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. - ** - * @param globalKTable the {@link GlobalKTable} to be joined with this stream - * @param keyValueMapper instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@link KStream} - * @return a {@link KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@link KStream} record + * In contrast to {@link #join(GlobalKTable, KeyValueMapper, ValueJoiner) inner-join}, all records from this stream + * will produce an output record (cf. below). + * The join is a primary key table lookup join with join attribute + * {@code keyValueMapper.map(stream.keyValue) == table.key}. + * "Table lookup join" means, that results are only computed if {@code KStream} records are processed. + * This is done by performing a lookup for matching records in the current internal {@link GlobalKTable} + * state. + * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} + * state and will not produce any result records. + *

      + * For each {@code KStream} record whether or not it finds a corresponding record in {@link GlobalKTable} the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. + * If no {@link GlobalKTable} record was found during lookup, a {@code null} value will be provided to + * {@link ValueJoiner}. + * The key of the result record is the same as this {@code KStream}. + * If an {@code KStream} input record key or value is {@code null} the record will not be included in the join + * operation and thus no output record will be added to the resulting {@code KStream}. + * + * @param globalKTable the {@link GlobalKTable} to be joined with this stream + * @param keyValueMapper instance of {@link KeyValueMapper} used to map from the (key, value) of this stream + * to the key of the {@link GlobalKTable} + * @param valueJoiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param the key type of {@link GlobalKTable} + * @param the value type of the {@link GlobalKTable} + * @param the value type of the resulting {@code KStream} + * @return a {@code KStream} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one output for each input {@code KStream} record * @see #join(GlobalKTable, KeyValueMapper, ValueJoiner) */ KStream leftJoin(final GlobalKTable globalKTable, final KeyValueMapper keyValueMapper, final ValueJoiner valueJoiner); - /** - * Join records of this stream with {@link GlobalKTable}'s records using non-windowed inner equi join - * The join is a primary key table lookup join with join attribute {@code keyValueMapper.map(stream.keyValue) == table.key}. - * "Table lookup join" means, that results are only computed if {@link KStream} records are processed. - * This is done by performing a lookup for matching records in the current internal - * {@link GlobalKTable} state. - * In contrast, processing {@link GlobalKTable} input records will only update the internal {@link GlobalKTable} state and - * will not produce any result records. - *

      - * For each {@link KStream} record that finds an corresponding record in {@link GlobalKTable} the provided - * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. - * The key of the result record is the same as the key of this {@link KStream} - * If an {@link KStream} input record key or value is {@code null} the record will not be included in the join - * operation and thus no output record will be added to the resulting {@link KStream}. - *

      - - * @param globalKTable the {@link GlobalKTable} to be joined with this stream - * @param keyValueMapper instance of {@link KeyValueMapper} used to map from the (key, value) of this stream - * to the key of the {@link GlobalKTable} - * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records - * @param the key type of {@link GlobalKTable} - * @param the value type of the {@link GlobalKTable} - * @param the value type of the resulting {@link KStream} - * @return a {@link KStream} that contains join-records for each key and values computed by the given - * {@link ValueJoiner}, one output for each input {@link KStream} record - * @see #leftJoin(KStream, ValueJoiner, JoinWindows) - */ - KStream join(final GlobalKTable globalKTable, - final KeyValueMapper keyValueMapper, - final ValueJoiner joiner); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index aecd8abd744d3..0a3304dc0cf3e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -19,15 +19,20 @@ import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.internals.GlobalKTableImpl; import org.apache.kafka.streams.kstream.internals.KStreamImpl; import org.apache.kafka.streams.kstream.internals.KTableImpl; import org.apache.kafka.streams.kstream.internals.KTableSource; import org.apache.kafka.streams.kstream.internals.KTableSourceValueGetterSupplier; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.TopologyBuilder; -import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.QueryableStoreType; import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreSupplier; import java.util.Collections; @@ -35,113 +40,139 @@ import java.util.regex.Pattern; /** - * {@link KStreamBuilder} is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL - * for users to specify computational logic and translates the given logic to a {@link org.apache.kafka.streams.processor.internals.ProcessorTopology}. + * {@code KStreamBuilder} provide the high-level Kafka Streams DSL to specify a Kafka Streams topology. + * + * @see TopologyBuilder + * @see KStream + * @see KTable + * @see GlobalKTable */ public class KStreamBuilder extends TopologyBuilder { private final AtomicInteger index = new AtomicInteger(0); /** - * Create a new {@link KStreamBuilder} instance. - */ - public KStreamBuilder() { - super(); - } - - /** - * Create a {@link KStream} instance from the specified topics. - * The default deserializers specified in the config are used. + * Create a {@link KStream} from the specified topics. + * The default {@code "auto.offset.reset"} strategy and default key and value deserializers as specified in the + * {@link StreamsConfig config} are used. + *

      + * If multiple topics are specified there is no ordering guarantee for records from different topics. *

      - * If multiple topics are specified there are nor ordering guaranteed for records from different topics. + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param topics the topic names; must contain at least one topic name + * @param topics the topic names; must contain at least one topic name * @return a {@link KStream} for the specified topics */ - public KStream stream(String... topics) { + public KStream stream(final String... topics) { return stream(null, null, null, topics); } - /** - * Create a {@link KStream} instance from the specified topics. - * The default deserializers specified in the config are used. + * Create a {@link KStream} from the specified topics. + * The default key and value deserializers as specified in the {@link StreamsConfig config} are used. + *

      + * If multiple topics are specified there is no ordering guarantee for records from different topics. *

      - * If multiple topics are specified there are nor ordering guaranteed for records from different topics. + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest - * @param topics the topic names; must contain at least one topic name + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topics if no valid committed + * offsets are available + * @param topics the topic names; must contain at least one topic name * @return a {@link KStream} for the specified topics */ - public KStream stream(AutoOffsetReset offsetReset, String... topics) { + public KStream stream(final AutoOffsetReset offsetReset, + final String... topics) { return stream(offsetReset, null, null, topics); } - /** - * Create a {@link KStream} instance from the specified Pattern. - * The default deserializers specified in the config are used. + * Create a {@link KStream} from the specified topic pattern. + * The default {@code "auto.offset.reset"} strategy and default key and value deserializers as specified in the + * {@link StreamsConfig config} are used. + *

      + * If multiple topics are matched by the specified pattern, the created {@link KStream} will read data from all of + * them and there is no ordering guarantee between records from different topics. *

      - * If multiple topics are matched by the specified pattern, the created stream will read data from all of them, - * and there is no ordering guarantee between records from different topics + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param topicPattern the Pattern to match for topic names + * @param topicPattern the pattern to match for topic names * @return a {@link KStream} for topics matching the regex pattern. */ - public KStream stream(Pattern topicPattern) { + public KStream stream(final Pattern topicPattern) { return stream(null, null, null, topicPattern); } /** - * Create a {@link KStream} instance from the specified Pattern. - * The default deserializers specified in the config are used. + * Create a {@link KStream} from the specified topic pattern. + * The default key and value deserializers as specified in the {@link StreamsConfig config} are used. *

      - * If multiple topics are matched by the specified pattern, the created stream will read data from all of them, - * and there is no ordering guarantee between records from different topics + * If multiple topics are matched by the specified pattern, the created {@link KStream} will read data from all of + * them and there is no ordering guarantee between records from different topics. + *

      + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest - * @param topicPattern the Pattern to match for topic names + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the matched topics if no valid committed + * offsets are available + * @param topicPattern the pattern to match for topic names * @return a {@link KStream} for topics matching the regex pattern. */ - public KStream stream(AutoOffsetReset offsetReset, Pattern topicPattern) { + public KStream stream(final AutoOffsetReset offsetReset, final Pattern topicPattern) { return stream(offsetReset, null, null, topicPattern); } - - /** - * Create a {@link KStream} instance from the specified topics. + * Create a {@link KStream} from the specified topics. + * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used. *

      - * If multiple topics are specified there are nor ordering guaranteed for records from different topics. + * If multiple topics are specified there is no ordering guarantee for records from different topics. + *

      + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param keySerde key serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param topics the topic names; must contain at least one topic name + * @param keySerde key serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param topics the topic names; must contain at least one topic name * @return a {@link KStream} for the specified topics */ - public KStream stream(Serde keySerde, Serde valSerde, String... topics) { + public KStream stream(final Serde keySerde, final Serde valSerde, final String... topics) { return stream(null, keySerde, valSerde, topics); } /** - * Create a {@link KStream} instance from the specified topics. + * Create a {@link KStream} from the specified topics. *

      - * If multiple topics are specified there are nor ordering guaranteed for records from different topics. - * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest + * If multiple topics are specified there is no ordering guarantee for records from different topics. + *

      + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param keySerde key serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param topics the topic names; must contain at least one topic name + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topics if no valid committed + * offsets are available + * @param keySerde key serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param topics the topic names; must contain at least one topic name * @return a {@link KStream} for the specified topics */ - public KStream stream(AutoOffsetReset offsetReset, Serde keySerde, Serde valSerde, String... topics) { - String name = newName(KStreamImpl.SOURCE_NAME); + public KStream stream(final AutoOffsetReset offsetReset, + final Serde keySerde, + final Serde valSerde, + final String... topics) { + final String name = newName(KStreamImpl.SOURCE_NAME); addSource(offsetReset, name, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topics); @@ -150,38 +181,53 @@ public KStream stream(AutoOffsetReset offsetReset, Serde keySerd /** - * Create a {@link KStream} instance from the specified Pattern. + * Create a {@link KStream} from the specified topic pattern. + * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used. + *

      + * If multiple topics are matched by the specified pattern, the created {@link KStream} will read data from all of + * them and there is no ordering guarantee between records from different topics. *

      - * If multiple topics are matched by the specified pattern, the created stream will read data from all of them, - * and there is no ordering guarantee between records from different topics. + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param keySerde key serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param topicPattern the Pattern to match for topic names - * @return a {@link KStream} for the specified topics + * @param keySerde key serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param topicPattern the pattern to match for topic names + * @return a {@link KStream} for topics matching the regex pattern. */ - public KStream stream(Serde keySerde, Serde valSerde, Pattern topicPattern) { + public KStream stream(final Serde keySerde, + final Serde valSerde, + final Pattern topicPattern) { return stream(null, keySerde, valSerde, topicPattern); } /** - * Create a {@link KStream} instance from the specified Pattern. + * Create a {@link KStream} from the specified topic pattern. *

      - * If multiple topics are matched by the specified pattern, the created stream will read data from all of them, - * and there is no ordering guarantee between records from different topics. + * If multiple topics are matched by the specified pattern, the created {@link KStream} will read data from all of + * them and there is no ordering guarantee between records from different topics. + *

      + * Note that the specified input topics must be partitioned by key. + * If this is not the case it is the user's responsibility to repartition the date before any key based operation + * (like aggregation or join) is applied to the returned {@link KStream}. * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest - * @param keySerde key serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to read this source {@link KStream}, - * if not specified the default serde defined in the configs will be used - * @param topicPattern the Pattern to match for topic names - * @return a {@link KStream} for the specified topics + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the matched topics if no valid committed + * offsets are available + * @param keySerde key serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param topicPattern the pattern to match for topic names + * @return a {@link KStream} for topics matching the regex pattern. */ - public KStream stream(AutoOffsetReset offsetReset, Serde keySerde, Serde valSerde, Pattern topicPattern) { - String name = newName(KStreamImpl.SOURCE_NAME); + public KStream stream(final AutoOffsetReset offsetReset, + final Serde keySerde, + final Serde valSerde, + final Pattern topicPattern) { + final String name = newName(KStreamImpl.SOURCE_NAME); addSource(offsetReset, name, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topicPattern); @@ -189,71 +235,152 @@ public KStream stream(AutoOffsetReset offsetReset, Serde keySerd } /** - * Create a {@link KTable} instance for the specified topic. - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The default deserializers specified in the config are used. - * The resulting {@link KTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. + * Create a {@link KTable} for the specified topic. + * The default {@code "auto.offset.reset"} strategy and default key and value deserializers as specified in the + * {@link StreamsConfig config} are used. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * Note that the specified input topic must be partitioned by key. + * If this is not the case the returned {@link KTable} will be corrupted. + *

      + * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest - * @param topic the topic name; cannot be null - * @param storeName the state store name used if this KTable is materialized, can be null if materialization not expected - * @return a {@link KTable} for the specified topics + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link KTable} for the specified topic */ - public KTable table(AutoOffsetReset offsetReset, String topic, final String storeName) { - return table(offsetReset, null, null, topic, storeName); + public KTable table(final String topic, + final String storeName) { + return table(null, null, null, topic, storeName); } /** - * Create a {@link KTable} instance for the specified topic. - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The default deserializers specified in the config are used. - * The resulting {@link KTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. + * Create a {@link KTable} for the specified topic. + * The default key and value deserializers as specified in the {@link StreamsConfig config} are used. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * Note that the specified input topic must be partitioned by key. + * If this is not the case the returned {@link KTable} will be corrupted. + *

      + * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. * - * @param topic the topic name; cannot be null - * @param storeName the state store name used if this KTable is materialized, can be null if materialization not expected - * @return a {@link KTable} for the specified topics + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed + * offsets are available + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link KTable} for the specified topic */ - public KTable table(String topic, final String storeName) { - return table(null, null, null, topic, storeName); + public KTable table(final AutoOffsetReset offsetReset, + final String topic, + final String storeName) { + return table(offsetReset, null, null, topic, storeName); } - /** - * Create a {@link KTable} instance for the specified topic. - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The resulting {@link KTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. + * Create a {@link KTable} for the specified topic. + * The default {@code "auto.offset.reset"} strategy as specified in the {@link StreamsConfig config} is used. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * Note that the specified input topic must be partitioned by key. + * If this is not the case the returned {@link KTable} will be corrupted. + *

      + * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name; cannot be null - * @param storeName the state store name used for the materialized KTable - * @return a {@link KTable} for the specified topics + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link KTable} for the specified topic */ - public KTable table(Serde keySerde, Serde valSerde, String topic, final String storeName) { + public KTable table(final Serde keySerde, + final Serde valSerde, + final String topic, + final String storeName) { return table(null, keySerde, valSerde, topic, storeName); } /** - * Create a {@link KTable} instance for the specified topic. - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The resulting {@link KTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. + * Create a {@link KTable} for the specified topic. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * Note that the specified input topic must be partitioned by key. + * If this is not the case the returned {@link KTable} will be corrupted. + *

      + * The resulting {@link KTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key); // key must be local (application state is shared over all running Kafka Streams instances)
      +     * }
      + * For non-local keys, a custom RPC mechanism must be implemented using {@link KafkaStreams#allMetadata()} to + * query the value of the key on a parallel running instance of your Kafka Streams application. * - * @param offsetReset the auto offset reset policy to use for this stream if no committed offsets available; acceptable values are earliest or latest - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name; cannot be null - * @param storeName the state store name used if this KTable is materialized, can be null if materialization not expected - * @return a {@link KTable} for the specified topics + * @param offsetReset the {@code "auto.offset.reset"} policy to use for the specified topic if no valid committed + * offsets are available + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link KTable} for the specified topic */ - public KTable table(AutoOffsetReset offsetReset, Serde keySerde, Serde valSerde, String topic, final String storeName) { + public KTable table(final AutoOffsetReset offsetReset, + final Serde keySerde, + final Serde valSerde, + final String topic, + final String storeName) { final String source = newName(KStreamImpl.SOURCE_NAME); final String name = newName(KTableImpl.SOURCE_NAME); final ProcessorSupplier processorSupplier = new KTableSource<>(storeName); @@ -265,7 +392,7 @@ public KTable table(AutoOffsetReset offsetReset, Serde keySerde, // only materialize the KTable into a state store if the storeName is not null if (storeName != null) { - StateStoreSupplier storeSupplier = new RocksDBKeyValueStoreSupplier<>(storeName, + final StateStoreSupplier storeSupplier = new RocksDBKeyValueStoreSupplier<>(storeName, keySerde, valSerde, false, @@ -279,22 +406,70 @@ public KTable table(AutoOffsetReset offsetReset, Serde keySerde, return kTable; } + /** + * Create a {@link GlobalKTable} for the specified topic. + * The default key and value deserializers as specified in the {@link StreamsConfig config} are used. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key);
      +     * }
      + * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"} + * regardless of the specified value in {@link StreamsConfig}. + * + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link GlobalKTable} for the specified topic + */ + public GlobalKTable globalTable(final String topic, + final String storeName) { + return globalTable(null, null, topic, storeName); + } /** - * Create a new {@link GlobalKTable} instance for the specified topic. - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The resulting {@link GlobalKTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name; cannot be null - * @param storeName the state store name used - * @return a {@link GlobalKTable} for the specified topics + * Create a {@link GlobalKTable} for the specified topic. + * The default key and value deserializers as specified in the {@link StreamsConfig config} are used. + * Input {@link KeyValue records} with {@code null} key will be dropped. + *

      + * The resulting {@link GlobalKTable} will be materialized in a local {@link KeyValueStore} with the given + * {@code storeName}. + * However, no internal changelog topic is created since the original input topic can be used for recovery (cf. + * methods of {@link KGroupedStream} and {@link KGroupedTable} that return a {@link KTable}). + *

      + * To query the local {@link KeyValueStore} it must be obtained via + * {@link KafkaStreams#store(String, QueryableStoreType) KafkaStreams#store(...)}: + *

      {@code
      +     * KafkaStreams streams = ...
      +     * ReadOnlyKeyValueStore localStore = streams.store(storeName, QueryableStoreTypes.keyValueStore());
      +     * String key = "some-key";
      +     * Long valueForKey = localStore.get(key);
      +     * }
      + * Note that {@link GlobalKTable} always applies {@code "auto.offset.reset"} strategy {@code "earliest"} + * regardless of the specified value in {@link StreamsConfig}. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name; cannot be {@code null} + * @param storeName the state store name; cannot be {@code null} + * @return a {@link GlobalKTable} for the specified topic */ @SuppressWarnings("unchecked") - public GlobalKTable globalTable(final Serde keySerde, final Serde valSerde, final String topic, final String storeName) { + public GlobalKTable globalTable(final Serde keySerde, + final Serde valSerde, + final String topic, + final String storeName) { final String sourceName = newName(KStreamImpl.SOURCE_NAME); final String processorName = newName(KTableImpl.SOURCE_NAME); final KTableSource tableSource = new KTableSource<>(storeName); @@ -315,41 +490,27 @@ public GlobalKTable globalTable(final Serde keySerde, final Serd } /** - * Create a new {@link GlobalKTable} instance for the specified topic using the default key and value {@link Serde}s - * Record keys of the topic should never by null, otherwise an exception will be thrown at runtime. - * The resulting {@link GlobalKTable} will be materialized in a local state store with the given store name. - * However, no new changelog topic is created in this case since the underlying topic acts as one. - * - * @param topic the topic name; cannot be null - * @param storeName the state store name used if this KTable is materialized, can be null if materialization not expected - * @return a {@link GlobalKTable} for the specified topics - */ - public GlobalKTable globalTable(final String topic, final String storeName) { - return globalTable(null, null, topic, storeName); - } - - /** - * Create a new instance of {@link KStream} by merging the given streams. + * Create a new instance of {@link KStream} by merging the given {@link KStream}s. *

      - * There are nor ordering guaranteed for records from different streams. + * There is no ordering guarantee for records from different {@link KStream}s. * - * @param streams the instances of {@link KStream} to be merged + * @param streams the {@link KStream}s to be merged * @return a {@link KStream} containing all records of the given streams */ - public KStream merge(KStream... streams) { + public KStream merge(final KStream... streams) { return KStreamImpl.merge(this, streams); } /** + * This function is only for internal usage only and should not be called. + *

      * Create a unique processor name used for translation into the processor topology. - * This function is only for internal usage. * - * @param prefix processor name prefix + * @param prefix processor name prefix * @return a new unique name */ - public String newName(String prefix) { + public String newName(final String prefix) { return prefix + String.format("%010d", index.getAndIncrement()); } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index d95af0e8aa82f..eebac8fabc24f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -17,378 +17,799 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.clients.producer.internals.DefaultPartitioner; import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.internals.WindowedSerializer; +import org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner; import org.apache.kafka.streams.processor.StreamPartitioner; -import org.apache.kafka.clients.producer.internals.DefaultPartitioner; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; /** - * {@link KTable} is an abstraction of a changelog stream from a primary-keyed table. - * Each record in this stream is an update on the primary-keyed table with the record key as the primary key. + * {@code KTable} is an abstraction of a changelog stream from a primary-keyed table. + * Each record in this changelog stream is an update on the primary-keyed table with the record key as the primary key. *

      - * A {@link KTable} is either defined from one or multiple Kafka topics that are consumed message by message or - * the result of a {@link KTable} transformation. An aggregation of a {@link KStream} also yields a {@link KTable}. + * A {@code KTable} is either {@link KStreamBuilder#table(String, String) defined from a single Kafka topic} that is + * consumed message by message or the result of a {@code KTable} transformation. + * An aggregation of a {@link KStream} also yields a {@code KTable}. *

      - * A {@link KTable} can be transformed record by record, joined with another {@link KTable} or {@link KStream}, or - * can be re-partitioned and aggregated into a new {@link KTable}. + * A {@code KTable} can be transformed record by record, joined with another {@code KTable} or {@link KStream}, or + * can be re-partitioned and aggregated into a new {@code KTable}. + *

      + * Some {@code KTable}s have an internal state (a {@link ReadOnlyKeyValueStore}) and are therefore queryable via the + * interactive queries API. + * For example: + *

      {@code
      + *     final KTable table = ...
      + *     ...
      + *     final KafkaStreams streams = ...;
      + *     streams.start()
      + *     ...
      + *     final String queryableStoreName = table.getStoreName(); // returns null if KTable is not queryable
      + *     ReadOnlyKeyValueStore view = streams.store(queryableStoreName, QueryableStoreTypes.keyValueStore());
      + *     view.get(key);
      + *}
      * * @param Type of primary keys * @param Type of value changes - * * @see KStream + * @see KGroupedTable + * @see GlobalKTable + * @see KStreamBuilder#table(String, String) */ @InterfaceStability.Unstable public interface KTable { /** - * Create a new instance of {@link KTable} that consists of all elements of this stream which satisfy a predicate. - * - * @param predicate the instance of {@link Predicate} - * - * @return a {@link KTable} that contains only those records that satisfy the given predicate + * Create a new {@code KTable} that consists of all records of this {@code KTable} which satisfy the given + * predicate. + * All records that do not satisfy the predicate are dropped. + * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the + * result {@code KTable}. + * This is a stateless record-by-record operation. + *

      + * Note that {@code filter} for a changelog stream works different to {@link KStream#filter(Predicate) + * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records) + * have delete semantics. + * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded + * directly if required (i.e., if there is anything to be deleted). + * Furthermore, for each record that gets dropped (i.e., dot not satisfied the given predicate) a tombstone record + * is forwarded. + * + * @param predicate a filter {@link Predicate} that is applied to each record + * @return a {@code KTable} that contains only those records that satisfy the given predicate + * @see #filterNot(Predicate) */ - KTable filter(Predicate predicate); + KTable filter(final Predicate predicate); /** - * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate. - * - * @param predicate the instance of {@link Predicate} - * - * @return a {@link KTable} that contains only those records that do not satisfy the given predicate + * Create a new {@code KTable} that consists all records of this {@code KTable} which do not satisfy the + * given predicate. + * All records that do satisfy the predicate are dropped. + * For each {@code KTable} update the filter is evaluated on the update record to produce an update record for the + * result {@code KTable}. + * This is a stateless record-by-record operation. + *

      + * Note that {@code filterNot} for a changelog stream works different to {@link KStream#filterNot(Predicate) + * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records) + * have delete semantics. + * Thus, for tombstones the provided filter predicate is not evaluated but the tombstone record is forwarded + * directly if required (i.e., if there is anything to be deleted). + * Furthermore, for each record that gets dropped (i.e., does satisfy the given predicate) a tombstone record is + * forwarded. + * + * @param predicate a filter {@link Predicate} that is applied to each record + * @return a {@code KTable} that contains only those records that do not satisfy the given predicate + * @see #filter(Predicate) */ - KTable filterNot(Predicate predicate); + KTable filterNot(final Predicate predicate); /** - * Create a new instance of {@link KTable} by transforming the value of each element in this stream into a new value in the new stream. - * - * @param mapper the instance of {@link ValueMapper} - * @param the value type of the new stream - * - * @return a {@link KTable} that contains records with unmodified keys and new values of different type + * Create a new {@code KTable} by transforming the value of each record in this {@code KTable} into a new value + * (with possible new type)in the new {@code KTable}. + * For each {@code KTable} update the provided {@link ValueMapper} is applied to the value of the update record and + * computes a new value for it, resulting in an update record for the result {@code KTable}. + * Thus, an input record {@code } can be transformed into an output record {@code }. + * This is a stateless record-by-record operation. + *

      + * The example below counts the number of token of the value string. + *

      {@code
      +     * KTable inputTable = builder.table("topic");
      +     * KTable outputTable = inputTable.mapValue(new ValueMapper {
      +     *     Integer apply(String value) {
      +     *         return value.split(" ").length;
      +     *     }
      +     * });
      +     * }
      + *

      + * This operation preserves data co-location with respect to the key. + * Thus, no internal data redistribution is required if a key based operator (like a join) is applied to + * the result {@code KTable}. + *

      + * Note that {@code mapValues} for a changelog stream works different to {@link KStream#mapValues(ValueMapper) + * record stream filters}, because {@link KeyValue records} with {@code null} values (so-called tombstone records) + * have delete semantics. + * Thus, for tombstones the provided value-mapper is not evaluated but the tombstone record is forwarded directly to + * delete the corresponding record in the result {@code KTable}. + * + * @param mapper a {@link ValueMapper} that computes a new output value + * @param the value type of the result {@code KTable} + * @return a {@code KTable} that contains records with unmodified keys and new values (possibly of different type) */ - KTable mapValues(ValueMapper mapper); + KTable mapValues(final ValueMapper mapper); /** - * Print the elements of this stream to {@code System.out}. This function - * will use the generated name of the parent processor node to label the key/value pairs - * printed out to the console. - * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Print the update records of this {@code KTable} to {@code System.out}. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the console. + *

      + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code print()} is not applied to the internal state store and only called for each new {@code KTable} + * update record. */ void print(); /** - * Print the elements of this stream to {@code System.out}. This function - * will use the given name to label the key/value printed out to the console. - * - * @param streamName the name used to label the key/value pairs printed out to the console - * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Print the update records of this {@code KTable} to {@code System.out}. + * This function will use the given name to label the key/value pairs printed to the console. + *

      + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code print()} is not applied to the internal state store and only called for each new {@code KTable} + * update record. + * + * @param streamName the name used to label the key/value pairs printed to the console */ - void print(String streamName); + void print(final String streamName); /** - * Print the elements of this stream to {@code System.out} - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Print the update records of this {@code KTable} to {@code System.out}. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the console. + *

      + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code print()} is not applied to the internal state store and only called for each new {@code KTable} + * update record. + * + * @param keySerde key serde used to deserialize key if type is {@code byte[]}, + * @param valSerde value serde used to deserialize value if type is {@code byte[]}, */ - void print(Serde keySerde, Serde valSerde); + void print(final Serde keySerde, + final Serde valSerde); /** - * Print the elements of this stream to System.out - * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param streamName the name used to label the key/value pairs printed out to the console - * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Print the update records of this {@code KTable} to {@code System.out}. + * This function will use the given name to label the key/value pairs printed to the console. + *

      + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code print()} is not applied to the internal state store and only called for each new {@code KTable} + * update record. + * + * @param keySerde key serde used to deserialize key if type is {@code byte[]}, + * @param valSerde value serde used to deserialize value if type is {@code byte[]}, + * @param streamName the name used to label the key/value pairs printed to the console */ - void print(Serde keySerde, Serde valSerde, String streamName); + void print(final Serde keySerde, + final Serde valSerde, + final String streamName); /** - * Write the elements of this stream to a file at the given path using default serializers and deserializers. - * @param filePath name of file to write to + * Write the update records of this {@code KTable} to a file at the given path. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the file. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code writeAsText()} is not applied to the internal state store and only called for each new + * {@code KTable} update record. * - * Implementors will need to override {@code toString} for keys and values that are not of - * type {@link String}, {@link Integer} etc. to get meaningful information. + * @param filePath name of file to write to */ - void writeAsText(String filePath); + void writeAsText(final String filePath); /** - * Write the elements of this stream to a file at the given path. - * - * @param filePath name of file to write to + * Write the update records of this {@code KTable} to a file at the given path. + * This function will use the given name to label the key/value printed to the file. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code writeAsText()} is not applied to the internal state store and only called for each new + * {@code KTable} update record. + * + * @param filePath name of file to write to * @param streamName the name used to label the key/value pairs printed out to the console - * - * Implementors will need to override {@code toString} for keys and values that are not of - * type {@link String}, {@link Integer} etc. to get meaningful information. */ - void writeAsText(String filePath, String streamName); + void writeAsText(final String filePath, + final String streamName); /** - * Write the elements of this stream to a file at the given path. + * Write the update records of this {@code KTable} to a file at the given path. + * This function will use the generated name of the parent processor node to label the key/value pairs printed to + * the file. + *

      + * The provided serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code writeAsText()} is not applied to the internal state store and only called for each new + * {@code KTable} update record. * * @param filePath name of file to write to - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * - * Implementors will need to override {@code toString} for keys and values that are not of - * type {@link String}, {@link Integer} etc. to get meaningful information. + * @param keySerde key serde used to deserialize key if type is {@code byte[]}, + * @param valSerde value serde used to deserialize value if type is {@code byte[]}, */ - void writeAsText(String filePath, Serde keySerde, Serde valSerde); + void writeAsText(final String filePath, + final Serde keySerde, + final Serde valSerde); /** - * @param filePath name of file to write to - * @param streamName the name used to label the key/value pairs printed out to the console - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used + * Write the update records of this {@code KTable} to a file at the given path. + * This function will use the given name to label the key/value printed to the file. + *

      + * The default serde will be used to deserialize the key or value in case the type is {@code byte[]} before calling + * {@code toString()} on the deserialized object. + *

      + * Implementors will need to override {@code toString()} for keys and values that are not of type {@link String}, + * {@link Integer} etc. to get meaningful information. + *

      + * Note that {@code writeAsText()} is not applied to the internal state store and only called for each new + * {@code KTable} update record. * - * Implementors will need to override {@code toString} for keys and values that are not of - * type {@link String}, {@link Integer} etc. to get meaningful information. + * @param filePath name of file to write to + * @param streamName the name used to label the key/value pairs printed to the console + * @param keySerde key serde used to deserialize key if type is {@code byte[]}, + * @param valSerde value serde used to deserialize value if type is {@code byte[]}, */ - - void writeAsText(String filePath, String streamName, Serde keySerde, Serde valSerde); + void writeAsText(final String filePath, + final String streamName, + final Serde keySerde, + final Serde valSerde); /** - * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic - * using default serializers and deserializers and producer's {@link DefaultPartitioner}. - * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String, String)}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Perform an action on each update record of this {@code KTable}. + * Note that this is a terminal operation that returns void. + *

      + * Note that {@code foreach()} is not applied to the internal state store and only called for each new + * {@code KTable} update record. * - * @param topic the topic name - * @param storeName the state store name used for this KTable - * @return a new {@link KTable} that contains the exact same records as this {@link KTable} + * @param action an action to perform on each record */ - KTable through(String topic, String storeName); + void foreach(final ForeachAction action); /** - * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic using default serializers - * and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. - * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String, String)}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. + * Convert this changelog stream to a {@link KStream}. + *

      + * Note that this is a logical operation and only changes the "interpretation" of the stream, i.e., each record of + * this changelog stream is no longer treated as an update record (cf. {@link KStream} vs {@code KTable}). * - * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified producer's {@link DefaultPartitioner} will be used - * @param topic the topic name - * @param storeName the state store name used for this KTable - * @return a new {@link KTable} that contains the exact same records as this {@link KTable} + * @return a {@link KStream} that contains the same records as this {@code KTable} */ - KTable through(StreamPartitioner partitioner, String topic, String storeName); + KStream toStream(); /** - * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic. - * If {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} - * for the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used - * — otherwise producer's {@link DefaultPartitioner} is used. - * This is equivalent to calling {@link #to(Serde, Serde, String)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(Serde, Serde, String, String)}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. - * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name - * @param storeName the state store name used for this KTable - * @return a new {@link KTable} that contains the exact same records as this {@link KTable} + * Convert this changelog stream to a {@link KStream} using the given {@link KeyValueMapper} to select the new key. + *

      + * For example, you can compute the new key as the length of the value string. + *

      {@code
      +     * KTable table = builder.table("topic");
      +     * KTable keyedStream = table.toStream(new KeyValueMapper {
      +     *     Integer apply(String key, String value) {
      +     *         return value.length();
      +     *     }
      +     * });
      +     * }
      + * Setting a new key might result in an internal data redistribution if a key based operator (like an aggregation or + * join) is applied to the result {@link KStream}. + *

      + * This operation is equivalent to calling + * {@code table.}{@link #toStream() toStream}{@code ().}{@link KStream#selectKey(KeyValueMapper) selectKey(KeyValueMapper)}. + *

      + * Note that {@link #toStream()} is a logical operation and only changes the "interpretation" of the stream, i.e., + * each record of this changelog stream is no longer treated as an update record (cf. {@link KStream} vs {@code KTable}). + * + * @param mapper a {@link KeyValueMapper} that computes a new key for each record + * @param the new key type of the result stream + * @return a {@link KStream} that contains the same records as this {@code KTable} */ - KTable through(Serde keySerde, Serde valSerde, String topic, String storeName); + KStream toStream(final KeyValueMapper mapper); /** - * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic - * using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. - * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)} and - * {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(Serde, Serde, String, String)}. - * The resulting {@link KTable} will be materialized in a local state - * store with the given store name. Also a changelog topic named "${applicationId}-${storeName}-changelog" - * will be automatically created in Kafka for failure recovery, where "applicationID" - * is specified by the user in {@link org.apache.kafka.streams.StreamsConfig}. - * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key - * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used - * — otherwise {@link DefaultPartitioner} will be used - * @param topic the topic name - * @param storeName the state store name used for this KTable - * @return a new {@link KTable} that contains the exact same records as this {@link KTable} + * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default + * serializers and deserializers and producer's {@link DefaultPartitioner}. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + *

      + * This is equivalent to calling {@link #to(String) #to(someTopicName)} and + * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}. + *

      + * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf. + * {@link KStreamBuilder#table(String, String)}) + * + * @param topic the topic name + * @param storeName the state store name used for the result {@code KTable} + * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable} */ - KTable through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic, String storeName); + KTable through(final String topic, + final String storeName); /** - * Materialize this stream to a topic using default serializers specified in the config - * and producer's {@link DefaultPartitioner}. - * - * @param topic the topic name + * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using default + * serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of + * records to partitions. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + *

      + * This is equivalent to calling {@link #to(StreamPartitioner, String) #to(partitioner, someTopicName)} and + * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}. + *

      + * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf. + * {@link KStreamBuilder#table(String, String)}) + * + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link DefaultPartitioner} will be used + * @param topic the topic name + * @param storeName the state store name used for the result {@code KTable} + * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable} */ - void to(String topic); + KTable through(final StreamPartitioner partitioner, + final String topic, + final String storeName); /** - * Materialize this stream to a topic using default serializers specified in the config - * and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. - * - * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified producer's {@link DefaultPartitioner} will be used - * @param topic the topic name + * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + *

      + * If {@code keySerde} provides a {@link WindowedSerializer} for the key {@link WindowedStreamPartitioner} is + * used—otherwise producer's {@link DefaultPartitioner} is used. + *

      + * This is equivalent to calling {@link #to(Serde, Serde, String) #to(keySerde, valueSerde, someTopicName)} and + * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}. + *

      + * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf. + * {@link KStreamBuilder#table(String, String)}) + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name + * @param storeName the state store name used for the result {@code KTable} + * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable} */ - void to(StreamPartitioner partitioner, String topic); + KTable through(final Serde keySerde, Serde valSerde, + final String topic, + final String storeName); /** - * Materialize this stream to a topic. If {@code keySerde} provides a - * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key - * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used - * — otherwise producer's {@link DefaultPartitioner} is used. - * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param topic the topic name + * Materialize this changelog stream to a topic and creates a new {@code KTable} from the topic using a customizable + * {@link StreamPartitioner} to determine the distribution of records to partitions. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + *

      + * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String) + * #to(keySerde, valueSerde, partitioner, someTopicName)} and + * {@link KStreamBuilder#table(String, String) KStreamBuilder#table(someTopicName, queryableStoreName)}. + *

      + * The resulting {@code KTable} will be materialized in a local state store with the given store name (cf. + * {@link KStreamBuilder#table(String, String)}) + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key + * {@link WindowedStreamPartitioner} will be used—otherwise {@link DefaultPartitioner} will + * be used + * @param topic the topic name + * @param storeName the state store name used for the result {@code KTable} + * @return a {@code KTable} that contains the exact same (and potentially repartitioned) records as this {@code KTable} */ - void to(Serde keySerde, Serde valSerde, String topic); + KTable through(final Serde keySerde, + final Serde valSerde, + final StreamPartitioner partitioner, + final String topic, + final String storeName); /** - * Materialize this stream to a topic using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * Materialize this changelog stream to a topic using default serializers and deserializers and producer's + * {@link DefaultPartitioner}. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param partitioner the function used to determine how records are distributed among partitions of the topic, - * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key - * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used - * — otherwise {@link DefaultPartitioner} will be used - * @param topic the topic name + * @param topic the topic name */ - void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); + void to(final String topic); /** - * Convert this stream to a new instance of {@link KStream}. - * - * @return a {@link KStream} that contains the same records as this {@link KTable}; - * the records are no longer treated as updates on a primary-keyed table, - * but rather as normal key-value pairs in a record stream + * Materialize this changelog stream to a topic using default serializers and deserializers and a customizable + * {@link StreamPartitioner} to determine the distribution of records to partitions. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + * + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link DefaultPartitioner} will be used + * @param topic the topic name */ - KStream toStream(); + void to(final StreamPartitioner partitioner, + final String topic); /** - * Convert this stream to a new instance of {@link KStream} using the given {@link KeyValueMapper} to select - * the new key. - * - * @param mapper @param mapper the instance of {@link KeyValueMapper} - * @param the new key type + * Materialize this changelog stream to a topic. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + *

      + * If {@code keySerde} provides a {@link WindowedSerializer} for the key {@link WindowedStreamPartitioner} is + * used—otherwise producer's {@link DefaultPartitioner} is used. * - * @return a {@link KStream} that contains the transformed records from this {@link KTable}; - * the records are no longer treated as updates on a primary-keyed table, - * but rather as normal key-value pairs in a record stream + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name */ - KStream toStream(KeyValueMapper mapper); + void to(final Serde keySerde, + final Serde valSerde, + final String topic); /** - * Combine values of this stream with another {@link KTable} stream's elements of the same key using Inner Join. - * - * @param other the instance of {@link KTable} joined with this stream - * @param joiner the instance of {@link ValueJoiner} - * @param the value type of the other stream - * @param the value type of the new stream - * - * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, - * one for each matched record-pair with the same key + * Materialize this changelog stream to a topic using a customizable {@link StreamPartitioner} to determine the + * distribution of records to partitions. + * The specified topic should be manually created before it is used (i.e., before the Kafka Streams application is + * started). + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link WindowedSerializer} for the key + * {@link WindowedStreamPartitioner} will be used—otherwise {@link DefaultPartitioner} will + * be used + * @param topic the topic name */ - KTable join(KTable other, ValueJoiner joiner); + void to(final Serde keySerde, + final Serde valSerde, + final StreamPartitioner partitioner, + final String topic); /** - * Combine values of this stream with another {@link KTable} stream's elements of the same key using Outer Join. - * - * @param other the instance of {@link KTable} joined with this stream - * @param joiner the instance of {@link ValueJoiner} - * @param the value type of the other stream - * @param the value type of the new stream - * - * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, - * one for each matched record-pair with the same key + * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper} and default serializers + * and deserializers. + * Each {@link KeyValue} pair of this {@code KTable} is mapped to a new {@link KeyValue} pair by applying the + * provided {@link KeyValueMapper}. + * Re-grouping a {@code KTable} is required before an aggregation operator can be applied to the data + * (cf. {@link KGroupedTable}). + * The {@link KeyValueMapper} selects a new key and value (with should both have unmodified type). + * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedTable} + *

      + * Because a new key is selected, an internal repartitioning topic will be created in Kafka. + * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is + * an internally generated name, and "-repartition" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + *

      + * All data of this {@code KTable} will be redistributed through the repartitioning topic by writing all update + * records to and rereading all update records from it, such that the resulting {@link KGroupedTable} is partitioned + * on the new key. + *

      + * If the key or value type is changed, it is recommended to use {@link #groupBy(KeyValueMapper, Serde, Serde)} + * instead. + * + * @param selector a {@link KeyValueMapper} that computes a new grouping key and value to be aggregated + * @param the key type of the result {@link KGroupedTable} + * @param the value type of the result {@link KGroupedTable} + * @return a {@link KGroupedTable} that contains the re-grouped records of the original {@code KTable} */ - KTable outerJoin(KTable other, ValueJoiner joiner); + KGroupedTable groupBy(final KeyValueMapper> selector); /** - * Combine values of this stream with another {@link KTable} stream's elements of the same key using Left Join. - * - * @param other the instance of {@link KTable} joined with this stream - * @param joiner the instance of {@link ValueJoiner} - * @param the value type of the other stream - * @param the value type of the new stream - * - * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, - * one for each matched record-pair with the same key + * Re-groups the records of this {@code KTable} using the provided {@link KeyValueMapper}. + * Each {@link KeyValue} pair of this {@code KTable} is mapped to a new {@link KeyValue} pair by applying the + * provided {@link KeyValueMapper}. + * Re-grouping a {@code KTable} is required before an aggregation operator can be applied to the data + * (cf. {@link KGroupedTable}). + * The {@link KeyValueMapper} selects a new key and value (both with potentially different type). + * If the new record key is {@code null} the record will not be included in the resulting {@link KGroupedTable} + *

      + * Because a new key is selected, an internal repartitioning topic will be created in Kafka. + * This topic will be named "${applicationId}-XXX-repartition", where "applicationId" is user-specified in + * {@link StreamsConfig} via parameter {@link StreamsConfig#APPLICATION_ID_CONFIG APPLICATION_ID_CONFIG}, "XXX" is + * an internally generated name, and "-repartition" is a fixed suffix. + * You can retrieve all generated internal topic names via {@link KafkaStreams#toString()}. + *

      + * All data of this {@code KTable} will be redistributed through the repartitioning topic by writing all update + * records to and rereading all update records from it, such that the resulting {@link KGroupedTable} is partitioned + * on the new key. + * + * @param selector a {@link KeyValueMapper} that computes a new grouping key and value to be aggregated + * @param keySerde key serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param the key type of the result {@link KGroupedTable} + * @param the value type of the result {@link KGroupedTable} + * @return a {@link KGroupedTable} that contains the re-grouped records of the original {@code KTable} */ - KTable leftJoin(KTable other, ValueJoiner joiner); - + KGroupedTable groupBy(final KeyValueMapper> selector, + final Serde keySerde, + final Serde valueSerde); /** - * Group the records of this {@link KTable} using the provided {@link KeyValueMapper}. - * - * @param selector select the grouping key and value to be aggregated - * @param keySerde key serdes for materializing this stream, - * if not specified the default serdes defined in the configs will be used - * @param valueSerde value serdes for materializing this stream, - * if not specified the default serdes defined in the configs will be used - * @param the key type of the {@link KGroupedTable} - * @param the value type of the {@link KGroupedTable} - * - * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable} + * Join records of this {@code KTable} with another {@code KTable}'s records using non-windowed inner equi join. + * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. + * The result is an ever updating {@code KTable} that represents the current (i.e., processing time) result + * of the join. + *

      + * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a + * matching record in the current (i.e., processing time) internal state of the other {@code KTable}. + * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input + * {@code KTable} the result gets updated. + *

      + * For each {@code KTable} record that finds a corresponding record in the other {@code KTable} the provided + * {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. + * The key of the result record is the same as for both joining input records. + *

      + * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics. + * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded + * directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted). + *

      + * Input records with {@code null} key will be dropped and no join computation is performed. + *

      + * Example: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
      thisKTablethisStateotherKTableotherStateresult update record
      <K1:A><K1:A>
      <K1:A><K1:b><K1:b><K1:ValueJoiner(A,b)>
      <K1:C><K1:C><K1:b><K1:ValueJoiner(C,b)>
      <K1:C><K1:null><K1:null>
      + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * + * @param other the other {@code KTable} to be joined with this {@code KTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} + * @return a {@code KTable} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one for each matched record-pair with the same key + * @see #leftJoin(KTable, ValueJoiner) + * @see #outerJoin(KTable, ValueJoiner) */ - KGroupedTable groupBy(KeyValueMapper> selector, Serde keySerde, Serde valueSerde); + KTable join(final KTable other, + final ValueJoiner joiner); /** - * Group the records of this {@link KTable} using the provided {@link KeyValueMapper} and default serializers and deserializers. - * - * @param selector select the grouping key and value to be aggregated - * @param the key type of the {@link KGroupedTable} - * @param the value type of the {@link KGroupedTable} - * - * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable} + * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using + * non-windowed left equi join. + * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. + * In contrast to {@link #join(KTable, ValueJoiner) inner-join}, all records from left {@code KTable} will produce + * an output record (cf. below). + * The result is an ever updating {@code KTable} that represents the current (i.e., processing time) result + * of the join. + *

      + * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a + * matching record in the current (i.e., processing time) internal state of the other {@code KTable}. + * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input + * {@code KTable} the result gets updated. + *

      + * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. + * Additionally, for each record of left {@code KTable} that does not find a corresponding record in the + * right {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code rightValue = + * null} to compute a value (with arbitrary type) for the result record. + * The key of the result record is the same as for both joining input records. + *

      + * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics. + * For example, for left input tombstones the provided value-joiner is not called but a tombstone record is + * forwarded directly to delete a record in the result {@code KTable} if required (i.e., if there is anything to be + * deleted). + *

      + * Input records with {@code null} key will be dropped and no join computation is performed. + *

      + * Example: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
      thisKTablethisStateotherKTableotherStateresult update record
      <K1:A><K1:A><K1:ValueJoiner(A,null)>
      <K1:A><K1:b><K1:b><K1:ValueJoiner(A,b)>
      <K1:null><K1:b><K1:null>
      <K1:null>
      + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * + * @param other the other {@code KTable} to be joined with this {@code KTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} + * @return a {@code KTable} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of + * left {@code KTable} + * @see #join(KTable, ValueJoiner) + * @see #outerJoin(KTable, ValueJoiner) */ - KGroupedTable groupBy(KeyValueMapper> selector); + KTable leftJoin(final KTable other, + final ValueJoiner joiner); /** - * Perform an action on each element of {@link KTable}. - * Note that this is a terminal operation that returns void. - * - * @param action an action to perform on each element + * Join records of this {@code KTable} (left input) with another {@code KTable}'s (right input) records using + * non-windowed outer equi join. + * The join is a primary key join with join attribute {@code thisKTable.key == otherKTable.key}. + * In contrast to {@link #join(KTable, ValueJoiner) inner-join} or {@link #leftJoin(KTable, ValueJoiner) left-join}, + * all records from both input {@code KTable}s will produce an output record (cf. below). + * The result is an ever updating {@code KTable} that represents the current (i.e., processing time) result + * of the join. + *

      + * The join is computed by (1) updating the internal state of one {@code KTable} and (2) performing a lookup for a + * matching record in the current (i.e., processing time) internal state of the other {@code KTable}. + * This happens in a symmetric way, i.e., for each update of either {@code this} or the {@code other} input + * {@code KTable} the result gets updated. + *

      + * For each {@code KTable} record that finds a corresponding record in the other {@code KTable}'s state the + * provided {@link ValueJoiner} will be called to compute a value (with arbitrary type) for the result record. + * Additionally, for each record that does not find a corresponding record in the corresponding other + * {@code KTable}'s state the provided {@link ValueJoiner} will be called with {@code null} value for the + * corresponding other value to compute a value (with arbitrary type) for the result record. + * The key of the result record is the same as for both joining input records. + *

      + * Note that {@link KeyValue records} with {@code null} values (so-called tombstone records) have delete semantics. + * Thus, for input tombstones the provided value-joiner is not called but a tombstone record is forwarded directly + * to delete a record in the result {@code KTable} if required (i.e., if there is anything to be deleted). + *

      + * Input records with {@code null} key will be dropped and no join computation is performed. + *

      + * Example: + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + * + *
      thisKTablethisStateotherKTableotherStateresult update record
      <K1:A><K1:A><K1:ValueJoiner(A,null)>
      <K1:A><K1:b><K1:b><K1:ValueJoiner(A,b)>
      <K1:null><K1:b><K1:ValueJoiner(null,b)>
      <K1:null><K1:null>
      + * Both input streams (or to be more precise, their underlying source topics) need to have the same number of + * partitions. + * + * @param other the other {@code KTable} to be joined with this {@code KTable} + * @param joiner a {@link ValueJoiner} that computes the join result for a pair of matching records + * @param the value type of the other {@code KTable} + * @param the value type of the result {@code KTable} + * @return a {@code KTable} that contains join-records for each key and values computed by the given + * {@link ValueJoiner}, one for each matched record-pair with the same key plus one for each non-matching record of + * both {@code KTable}s + * @see #join(KTable, ValueJoiner) + * @see #leftJoin(KTable, ValueJoiner) */ - void foreach(ForeachAction action); + KTable outerJoin(final KTable other, + final ValueJoiner joiner); /** - * Get the name of the local state store used for materializing this {@link KTable} - * @return the underlying state store name, or {@code null} if KTable does not have one + * Get the name of the local state store used for materializing this {@code KTable}. + * + * @return the underlying state store name, or {@code null} if this {@code KTable} is not materialized */ String getStoreName(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java index d6d1defb9fdd6..e2434e37692fb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -17,24 +17,25 @@ package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.KeyValue; /** - * The {@link KeyValueMapper} interface for mapping a {@link org.apache.kafka.streams.KeyValue key-value pair} to a new - * value of arbitrary type. For example, it can be used to + * The {@code KeyValueMapper} interface for mapping a {@link KeyValue key-value pair} to a new value of arbitrary type. + * For example, it can be used to *

        - *
      • map from an input {@link org.apache.kafka.streams.KeyValue key-value pair} to an output - * {@link org.apache.kafka.streams.KeyValue key-value pair} with different key and/or value type (for this case - * output type {@code VR == }{@link org.apache.kafka.streams.KeyValue KeyValue<NewKeyType,NewValueType>})
      • + *
      • map from an input {@link KeyValue} pair to an output {@link KeyValue} pair with different key and/or value type + * (for this case output type {@code VR == }{@link KeyValue KeyValue<NewKeyType,NewValueType>})
      • *
      • map from an input record to a new key (with arbitrary key type as specified by {@code VR})
      • *
      * This is a stateless record-by-record operation, i.e, {@link #apply(Object, Object)} is invoked individually for each - * record of a stream. - * {@link KeyValueMapper} is a generalization of {@link ValueMapper}. + * record of a stream (cf. {@link Transformer} for stateful record transformation). + * {@code KeyValueMapper} is a generalization of {@link ValueMapper}. * * @param key type * @param value type * @param mapped value type * @see ValueMapper + * @see Transformer * @see KStream#map(KeyValueMapper) * @see KStream#flatMap(KeyValueMapper) * @see KStream#selectKey(KeyValueMapper) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java index 5a70f21366559..f0df2c609516f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Merger.java @@ -29,7 +29,7 @@ public interface Merger { /** - * Compute a new aggregate from the key and two aggregates + * Compute a new aggregate from the key and two aggregates. * * @param aggKey the key of the record * @param aggOne the first aggregate diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java index 24563ba9c2cd0..cf1a1f335b2f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.KeyValue; /** - * The {@link Predicate} interface represents a predicate (boolean-valued function) of a - * {@link org.apache.kafka.streams.KeyValue key-value pair}. + * The {@code Predicate} interface represents a predicate (boolean-valued function) of a {@link KeyValue} pair. * This is a stateless record-by-record operation, i.e, {@link #test(Object, Object)} is invoked individually for each * record of a stream. * @@ -40,8 +40,7 @@ public interface Predicate { * * @param key the key of the record * @param value the value of the record - * @return {@code true} if the {@link org.apache.kafka.streams.KeyValue key-value pair} satisfies the - * predicate—{@code false} otherwise + * @return {@code true} if the {@link KeyValue} pair satisfies the predicate—{@code false} otherwise */ boolean test(final K key, final V value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java index d19c1eb1b68b1..5a273e4ee135f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java @@ -18,15 +18,15 @@ package org.apache.kafka.streams.kstream; import org.apache.kafka.common.annotation.InterfaceStability; - +import org.apache.kafka.streams.KeyValue; /** - * The {@link Reducer} interface for combining two values of the same type into a new value. + * The {@code Reducer} interface for combining two values of the same type into a new value. * In contrast to {@link Aggregator} the result type must be the same as the input type. *

      - * The provided values can be either original values from input {@link org.apache.kafka.streams.KeyValue KeyValue} pair - * records or be a previously computed result from {@link Reducer#apply(Object, Object)}. + * The provided values can be either original values from input {@link KeyValue} pair records or be a previously + * computed result from {@link Reducer#apply(Object, Object)}. *

      - * {@link Reducer} can be used to implement aggregation functions like sum, min, or max. + * {@code Reducer} can be used to implement aggregation functions like sum, min, or max. * * @param value type * @see KGroupedStream#reduce(Reducer, String) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java index f9a399a6451e1..943092365dffd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SessionWindows.java @@ -16,18 +16,17 @@ */ package org.apache.kafka.streams.kstream; - import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.processor.TimestampExtractor; /** * A session based window specification used for aggregating events into sessions. *

      * Sessions represent a period of activity separated by a defined gap of inactivity. - * Any events processed that fall within the inactivity gap of any existing sessions - * are merged into the existing sessions. If the event falls outside of the session gap - * then a new session will be created. + * Any events processed that fall within the inactivity gap of any existing sessions are merged into the existing sessions. + * If the event falls outside of the session gap then a new session will be created. *

      - * For example, If we have a session gap of 5 and the following data arrives: + * For example, if we have a session gap of 5 and the following data arrives: *

        * +--------------------------------------+
        * |    key    |    value    |    time    |
      @@ -39,13 +38,12 @@
        * |    A      |     3       |     20     |
        * +-----------+-------------+------------+
        * 
      - *

      - * We'd have 2 sessions for key A. 1 starting from time 10 and ending at time 12 and another - * starting and ending at time 20. The length of the session is driven by the timestamps of - * the data within the session + * We'd have 2 sessions for key A. + * One starting from time 10 and ending at time 12 and another starting and ending at time 20. + * The length of the session is driven by the timestamps of the data within the session. + * Thus, session windows are no fixed-size windows (c.f. {@link TimeWindows} and {@link JoinWindows}). *

      * If we then received another record: - *

      *

        * +--------------------------------------+
        * |    key    |    value    |    time    |
      @@ -53,53 +51,81 @@
        * |    A      |     4       |     16     |
        * +-----------+-------------+------------+
        * 
      - *

      * The previous 2 sessions would be merged into a single session with start time 10 and end time 20. * The aggregate value for this session would be the result of aggregating all 4 values. + *

      + * For time semantics, see {@link TimestampExtractor}. + * + * @see TimeWindows + * @see UnlimitedWindows + * @see JoinWindows + * @see KGroupedStream#count(SessionWindows, String) + * @see KGroupedStream#count(SessionWindows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#reduce(Reducer, SessionWindows, String) + * @see KGroupedStream#reduce(Reducer, SessionWindows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Merger, SessionWindows, org.apache.kafka.common.serialization.Serde, String) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Merger, SessionWindows, org.apache.kafka.common.serialization.Serde, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see TimestampExtractor */ @InterfaceStability.Unstable -public class SessionWindows { +public final class SessionWindows { private final long gapMs; private long maintainDurationMs; - private SessionWindows(final long gapMs, final long maintainDurationMs) { + private SessionWindows(final long gapMs) { this.gapMs = gapMs; - this.maintainDurationMs = maintainDurationMs; + maintainDurationMs = Windows.DEFAULT_MAINTAIN_DURATION_MS; } /** - * Create a new SessionWindows with the specified inactivity gap - * @param inactivityGapMs the gap of inactivity between sessions - * @return a new SessionWindows with the provided inactivity gap - * and default maintain duration + * Create a new window specification with the specified inactivity gap in milliseconds. + * + * @param inactivityGapMs the gap of inactivity between sessions in milliseconds + * @return a new window specification with default maintain duration of 1 day + * + * @throws IllegalArgumentException if {@code inactivityGapMs} is zero or negative */ public static SessionWindows with(final long inactivityGapMs) { - return new SessionWindows(inactivityGapMs, Windows.DEFAULT_MAINTAIN_DURATION); + if (inactivityGapMs <= 0) { + throw new IllegalArgumentException("Gap time (inactivityGapMs) cannot be zero or negative."); + } + return new SessionWindows(inactivityGapMs); } /** - * Set the window maintain duration in milliseconds of streams time. + * Set the window maintain duration (retention time) in milliseconds. * This retention time is a guaranteed lower bound for how long a window will be maintained. * - * @return itself + * @return itself + * @throws IllegalArgumentException if {@code durationMs} is smaller than window gap */ - public SessionWindows until(final long durationMs) { - this.maintainDurationMs = durationMs; + public SessionWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < gapMs) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be smaller than window gap."); + } + maintainDurationMs = durationMs; + return this; } /** - * @return the inactivityGap + * Return the specified gap for the session windows in milliseconds. + * + * @return the inactivity gap of the specified windows */ public long inactivityGap() { return gapMs; } /** - * @return the minimum amount of time a window will be maintained for. + * Return the window maintain duration (retention time) in milliseconds. + *

      + * For {@code SessionWindows} the maintain duration is at least as small as the window gap. + * + * @return the window maintain duration */ public long maintainMs() { - return maintainDurationMs; + return Math.max(maintainDurationMs, gapMs); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java index ef94cf9b0abfa..14af875bebcaa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -5,9 +5,9 @@ * 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 - * + *

      + * 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. @@ -17,120 +17,159 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.HashMap; import java.util.Map; /** - * The time-based window specifications used for aggregations. + * The fixed-size time-based window specifications used for aggregations. *

      - * The semantics of a time-based window are: Every T1 (advance) time-units, compute the aggregate total for T2 (size) time-units. + * The semantics of time-based aggregation windows are: Every T1 (advance) milliseconds, compute the aggregate total for + * T2 (size) milliseconds. *

        - *
      • If {@code advance < size} a hopping windows is defined:
        - * it discretize a stream into overlapping windows, which implies that a record maybe contained in one and or more "adjacent" windows.
      • + *
      • If {@code advance < size} a hopping windows is defined:
        + * it discretize a stream into overlapping windows, which implies that a record maybe contained in one and or + * more "adjacent" windows.
      • *
      • If {@code advance == size} a tumbling window is defined:
        - * it discretize a stream into non-overlapping windows, which implies that a record is only ever contained in one and only one tumbling window.
      • + * it discretize a stream into non-overlapping windows, which implies that a record is only ever contained in + * one and only one tumbling window. *
      + * Thus, the specified {@link TimeWindow}s are aligned to the epoch. + * Aligned to the epoch means, that the first window starts at timestamp zero. + * For example, hopping windows with size of 5000ms and advance of 3000ms, have window boundaries + * [0;5000),[3000;8000),... and not [1000;6000),[4000;9000),... or even something "random" like [1452;6452),[4452;9452),... + *

      + * For time semantics, see {@link TimestampExtractor}. + * + * @see SessionWindows + * @see UnlimitedWindows + * @see JoinWindows + * @see KGroupedStream#count(Windows, String) + * @see KGroupedStream#count(Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#reduce(Reducer, Windows, String) + * @see KGroupedStream#reduce(Reducer, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, String) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see TimestampExtractor */ -public class TimeWindows extends Windows { +@InterfaceStability.Unstable +public final class TimeWindows extends Windows { - /** - * The size of the window, i.e. how long a window lasts. - * The window size's effective time unit is determined by the semantics of the topology's - * configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. - */ - public final long size; + /** The size of the windows in milliseconds. */ + public final long sizeMs; /** - * The size of the window's advance interval, i.e. by how much a window moves forward relative - * to the previous one. The interval's effective time unit is determined by the semantics of - * the topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. + * The size of the window's advance interval in milliseconds, i.e., by how much a window moves forward relative to + * the previous one. */ - public final long advance; - + public final long advanceMs; - private TimeWindows(long size, long advance) { - super(); - if (size <= 0) { - throw new IllegalArgumentException("window size must be > 0 (you provided " + size + ")"); - } - this.size = size; - if (!(0 < advance && advance <= size)) { - throw new IllegalArgumentException( - String.format("advance interval (%d) must lie within interval (0, %d]", advance, size)); - } - this.advance = advance; + private TimeWindows(final long sizeMs, final long advanceMs) { + this.sizeMs = sizeMs; + this.advanceMs = advanceMs; } /** - * Returns a window definition with the given window size, and with the advance interval being - * equal to the window size. Think: [N * size, N * size + size), with N denoting the N-th - * window. + * Return a window definition with the given window size, and with the advance interval being equal to the window + * size. + * The time interval represented by the the N-th window is: {@code [N * size, N * size + size)}. + *

      + * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, non-overlapping windows. + * Tumbling windows are a special case of hopping windows with {@code advance == size}. * - * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, - * non-overlapping windows. Tumbling windows are a specialization of hopping windows. - * - * @param size The size of the window, with the requirement that size > 0. - * The window size's effective time unit is determined by the semantics of the - * topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. - * @return a new window definition + * @param sizeMs The size of the window in milliseconds + * @return a new window definition with default maintain duration of 1 day + * @throws IllegalArgumentException if the specified window size is zero or negative */ - public static TimeWindows of(long size) { - return new TimeWindows(size, size); + public static TimeWindows of(final long sizeMs) throws IllegalArgumentException { + if (sizeMs <= 0) { + throw new IllegalArgumentException("Window size (sizeMs) must be larger than zero."); + } + return new TimeWindows(sizeMs, sizeMs); } /** - * Returns a window definition with the original size, but advance ("hop") the window by the given - * interval, which specifies by how much a window moves forward relative to the previous one. - * Think: [N * advanceInterval, N * advanceInterval + size), with N denoting the N-th window. - * + * Return a window definition with the original size, but advance ("hop") the window by the given interval, which + * specifies by how much a window moves forward relative to the previous one. + * The time interval represented by the the N-th window is: {@code [N * advance, N * advance + size)}. + *

      * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. * - * @param interval The advance interval ("hop") of the window, with the requirement that - * 0 < interval ≤ size. The interval's effective time unit is - * determined by the semantics of the topology's configured - * {@link org.apache.kafka.streams.processor.TimestampExtractor}. - * @return a new window definition + * @param advanceMs The advance interval ("hop") in milliseconds of the window, with the requirement that + * {@code 0 < advanceMs ≤ sizeMs}. + * @return a new window definition with default maintain duration of 1 day + * @throws IllegalArgumentException if the advance interval is negative, zero, or larger-or-equal the window size */ - public TimeWindows advanceBy(long interval) { - return new TimeWindows(this.size, interval); + public TimeWindows advanceBy(final long advanceMs) { + if (advanceMs <= 0 || advanceMs > sizeMs) { + throw new IllegalArgumentException(String.format("AdvanceMs must lie within interval (0, %d].", sizeMs)); + } + return new TimeWindows(sizeMs, advanceMs); } @Override - public Map windowsFor(long timestamp) { - long windowStart = (Math.max(0, timestamp - this.size + this.advance) / this.advance) * this.advance; - Map windows = new HashMap<>(); + public Map windowsFor(final long timestamp) { + long windowStart = (Math.max(0, timestamp - sizeMs + advanceMs) / advanceMs) * advanceMs; + final Map windows = new HashMap<>(); while (windowStart <= timestamp) { - TimeWindow window = new TimeWindow(windowStart, windowStart + this.size); + final TimeWindow window = new TimeWindow(windowStart, windowStart + sizeMs); windows.put(windowStart, window); - windowStart += this.advance; + windowStart += advanceMs; } return windows; } @Override public long size() { - return size; + return sizeMs; + } + + /** + * @param durationMs the window retention time + * @return itself + * @throws IllegalArgumentException if {@code duration} is smaller than the window size + */ + @Override + public TimeWindows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < sizeMs) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be smaller than the window size."); + } + super.until(durationMs); + return this; + } + + /** + * {@inheritDoc} + *

      + * For {@code TimeWindows} the maintain duration is at least as small as the window size. + * + * @return the window maintain duration + */ + @Override + public long maintainMs() { + return Math.max(super.maintainMs(), sizeMs); } @Override - public final boolean equals(Object o) { + public boolean equals(final Object o) { if (o == this) { return true; } if (!(o instanceof TimeWindows)) { return false; } - TimeWindows other = (TimeWindows) o; - return this.size == other.size && this.advance == other.advance; + final TimeWindows other = (TimeWindows) o; + return sizeMs == other.sizeMs && advanceMs == other.advanceMs; } @Override public int hashCode() { - int result = (int) (size ^ (size >>> 32)); - result = 31 * result + (int) (advance ^ (advance >>> 32)); + int result = (int) (sizeMs ^ (sizeMs >>> 32)); + result = 31 * result + (int) (advanceMs ^ (advanceMs >>> 32)); return result; } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index 95d822ad56e16..6d18477977e17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -17,48 +17,96 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TimestampExtractor; /** - * A stateful {@link Transformer} interface to transform a key-value pair into a new value. + * The {@code Transformer} interface for stateful mapping of an input record to zero, one, or multiple new output + * records (both key and value type can be altered arbitrarily). + * This is a stateful record-by-record operation, i.e, {@link #transform(Object, Object)} is invoked individually for + * each record of a stream and can access and modify a state that is available beyond a single call of + * {@link #transform(Object, Object)} (cf. {@link KeyValueMapper} for stateless record transformation). + * Additionally, the interface can be called in regular intervals based on the processing progress + * (cf. {@link #punctuate(long)}. + *

      + * Use {@link TransformerSupplier} to provide new instances of {@code Transformer} to Kafka Stream's runtime. + *

      + * If only a record's value should be modified {@link ValueTransformer} can be used. * - * @param key type - * @param value type - * @param return type + * @param key type + * @param value type + * @param {@link KeyValue} return type (both key and value type can be set + * arbitrarily) + * @see TransformerSupplier + * @see KStream#transform(TransformerSupplier, String...) + * @see ValueTransformer + * @see KStream#map(KeyValueMapper) + * @see KStream#flatMap(KeyValueMapper) */ +@InterfaceStability.Unstable public interface Transformer { /** - * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology - * that contains it is initialized. + * Initialize this transformer. + * This is called once per instance when the topology gets initialized. *

      - * If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should - * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to + * {@link ProcessorContext#schedule(long) schedule itself} for periodical calls (cf. {@link #punctuate(long)}), and + * to access attached {@link StateStore}s. + *

      + * Note, that {@link ProcessorContext} is updated in the background with the current record's meta data. + * Thus, it only contains valid record meta data when accessed within {@link #transform(Object, Object)}. * - * @param context the context; may not be null + * @param context the context */ void init(final ProcessorContext context); /** * Transform the record with the given key and value. + * Additionally, any {@link StateStore state} that is {@link KStream#transform(TransformerSupplier, String...) + * attached} to this operator can be accessed and modified + * arbitrarily (cf. {@link ProcessorContext#getStateStore(String)}). + *

      + * If more than one output record should be forwarded downstream {@link ProcessorContext#forward(Object, Object)}, + * {@link ProcessorContext#forward(Object, Object, int)}, and + * {@link ProcessorContext#forward(Object, Object, String)} can be used. + * If not record should be forwarded downstream, {@code transform} can return {@code null}. * * @param key the key for the record * @param value the value for the record - * @return new value; if null no key-value pair will be forwarded to down stream + * @return new {@link KeyValue} pair—if {@code null} no key-value pair will + * be forwarded to down stream */ R transform(final K key, final V value); /** - * Perform any periodic operations and possibly generate a key, if this processor {@link ProcessorContext#schedule(long) schedules itself} with the context - * during {@link #init(ProcessorContext) initialization}. + * Perform any periodic operations and possibly generate new {@link KeyValue} pairs if this processor + * {@link ProcessorContext#schedule(long) schedules itself} with the context during + * {@link #init(ProcessorContext) initialization}. + *

      + * To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)}, + * {@link ProcessorContext#forward(Object, Object, int)}, and + * {@link ProcessorContext#forward(Object, Object, String)} can be used. + *

      + * Note that {@code punctuate} is called based on stream time (i.e., time progresses with regard to + * timestamps return by the used {@link TimestampExtractor}) + * and not based on wall-clock time. * - * @param timestamp the stream time when this method is being called - * @return new value; if null it will not be forwarded to down stream + * @param timestamp the stream time when {@code punctuate} is being called + * @return must return {@code null}—otherwise, a {@link StreamsException exception} will be thrown */ R punctuate(final long timestamp); /** * Close this processor and clean up any resources. + *

      + * To generate new {@link KeyValue} pairs {@link ProcessorContext#forward(Object, Object)}, + * {@link ProcessorContext#forward(Object, Object, int)}, and + * {@link ProcessorContext#forward(Object, Object, String)} can be used. */ void close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java index 0341702dafe7d..f5385aeeba5c2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java @@ -17,15 +17,28 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; + /** - * A {@link TransformerSupplier} interface which can create one or more {@link Transformer} instances. + * A {@code TransformerSupplier} interface which can create one or more {@link Transformer} instances. + * + * @param key type + * @param value type + * @param {@link org.apache.kafka.streams.KeyValue KeyValue} return type (both key and value type can be set + * arbitrarily) + * @see Transformer + * @see KStream#transform(TransformerSupplier, String...) + * @see ValueTransformer + * @see ValueTransformerSupplier + * @see KStream#transformValues(ValueTransformerSupplier, String...) */ +@InterfaceStability.Unstable public interface TransformerSupplier { /** * Return a new {@link Transformer} instance. * - * @return a new {@link Transformer} instance + * @return a new {@link Transformer} instance */ Transformer get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 3dc6f654dcdec..14598e41e87bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -5,9 +5,9 @@ * 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 - * + *

      + * 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. @@ -17,65 +17,113 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.kstream.internals.UnlimitedWindow; +import org.apache.kafka.streams.processor.TimestampExtractor; import java.util.HashMap; import java.util.Map; /** - * The unlimited window specifications. + * The unlimited window specifications used for aggregations. + *

      + * An unlimited time window is also called landmark window. + * It has a fixed starting point while its window end is defined as infinite. + * With this regard, it is a fixed-size window with infinite window size. + *

      + * For time semantics, see {@link TimestampExtractor}. + * + * @see TimeWindows + * @see SessionWindows + * @see JoinWindows + * @see KGroupedStream#count(Windows, String) + * @see KGroupedStream#count(Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#reduce(Reducer, Windows, String) + * @see KGroupedStream#reduce(Reducer, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, String) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see TimestampExtractor */ -public class UnlimitedWindows extends Windows { +@InterfaceStability.Unstable +public final class UnlimitedWindows extends Windows { - private static final long DEFAULT_START_TIMESTAMP = 0L; + private static final long DEFAULT_START_TIMESTAMP_MS = 0L; /** The start timestamp of the window. */ - public final long start; + public final long startMs; - private UnlimitedWindows(long start) { - super(); - if (start < 0) { - throw new IllegalArgumentException("start must be > 0 (you provided " + start + ")"); - } - this.start = start; + private UnlimitedWindows(final long startMs) { + this.startMs = startMs; } /** * Return an unlimited window starting at timestamp zero. */ public static UnlimitedWindows of() { - return new UnlimitedWindows(DEFAULT_START_TIMESTAMP); + return new UnlimitedWindows(DEFAULT_START_TIMESTAMP_MS); } /** * Return a new unlimited window for the specified start timestamp. * - * @param start the window start time - * @return a new unlimited window that starts at {@code start} + * @param startMs the window start time + * @return a new unlimited window that starts at {@code startMs} + * @throws IllegalArgumentException if the start time is negative */ - public UnlimitedWindows startOn(long start) { - return new UnlimitedWindows(start); + public UnlimitedWindows startOn(final long startMs) throws IllegalArgumentException { + if (startMs < 0) { + throw new IllegalArgumentException("Window start time (startMs) cannot be negative."); + } + return new UnlimitedWindows(startMs); } @Override - public Map windowsFor(long timestamp) { + public Map windowsFor(final long timestamp) { // always return the single unlimited window // we cannot use Collections.singleMap since it does not support remove() - Map windows = new HashMap<>(); - if (timestamp >= start) { - windows.put(start, new UnlimitedWindow(start)); + final Map windows = new HashMap<>(); + if (timestamp >= startMs) { + windows.put(startMs, new UnlimitedWindow(startMs)); } return windows; } + /** + * {@inheritDoc} + * As unlimited windows have conceptually infinite size, this methods just returns {@link Long#MAX_VALUE}. + * + * @return the size of the specified windows which is {@link Long#MAX_VALUE} + */ @Override public long size() { return Long.MAX_VALUE; } + /** + * Throws an {@link IllegalArgumentException} because the retention time for unlimited windows is always infinite + * and cannot be changed. + * + * @throws IllegalArgumentException on every invocation + */ + @Override + public UnlimitedWindows until(final long durationMs) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be set for UnlimitedWindows."); + } + + /** + * {@inheritDoc} + * The retention time for unlimited windows in infinite and thus represented as {@link Long#MAX_VALUE}. + * + * @return the window retention time that is {@link Long#MAX_VALUE} + */ + @Override + public long maintainMs() { + return Long.MAX_VALUE; + } + @Override - public final boolean equals(Object o) { + public boolean equals(final Object o) { if (o == this) { return true; } @@ -84,17 +132,13 @@ public final boolean equals(Object o) { return false; } - UnlimitedWindows other = (UnlimitedWindows) o; - return this.start == other.start; + final UnlimitedWindows other = (UnlimitedWindows) o; + return startMs == other.startMs; } @Override public int hashCode() { - return (int) (start ^ (start >>> 32)); + return (int) (startMs ^ (startMs >>> 32)); } - @Override - public long maintainMs() { - return Long.MAX_VALUE; - } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index ab91bb4f7e327..94de484c28220 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * The {@link ValueJoiner} interface for joining two values into a new value of arbitrary type. + * The {@code ValueJoiner} interface for joining two values into a new value of arbitrary type. * This is a stateless operation, i.e, {@link #apply(Object, Object)} is invoked individually for each joining * record-pair of a {@link KStream}-{@link KStream}, {@link KStream}-{@link KTable}, or {@link KTable}-{@link KTable} * join. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java index 5099ac71e2ac3..63e0cc471aaca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -19,16 +19,17 @@ import org.apache.kafka.common.annotation.InterfaceStability; /** - * The {@link ValueMapper} interface for mapping a value to a new value of arbitrary type. + * The {@code ValueMapper} interface for mapping a value to a new value of arbitrary type. * This is a stateless record-by-record operation, i.e, {@link #apply(Object)} is invoked individually for each record - * of a stream. - * Thus, if {@link ValueMapper} is applied to a {@link org.apache.kafka.streams.KeyValue key-value pair} record the - * record's key is preserved. + * of a stream (cf. {@link ValueTransformer} for stateful value transformation). + * If {@code ValueMapper} is applied to a {@link org.apache.kafka.streams.KeyValue key-value pair} record the record's + * key is preserved. * If a record's key and value should be modified {@link KeyValueMapper} can be used. * * @param value type * @param mapped value type * @see KeyValueMapper + * @see ValueTransformer * @see KStream#mapValues(ValueMapper) * @see KStream#flatMapValues(ValueMapper) * @see KTable#mapValues(ValueMapper) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index 063c35233999c..b2008f5ba1f4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -17,46 +17,96 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TimestampExtractor; /** - * A stateful {@link ValueTransformer} interface to transform a value into a new value. + * The {@code ValueTransformer} interface for stateful mapping of a value to a new value (with possible new type). + * This is a stateful record-by-record operation, i.e, {@link #transform(Object)} is invoked individually for each + * record of a stream and can access and modify a state that is available beyond a single call of + * {@link #transform(Object)} (cf. {@link ValueMapper} for stateless value transformation). + * Additionally, the interface can be called in regular intervals based on the processing progress + * (cf. {@link #punctuate(long)}. + * If {@code ValueTransformer} is applied to a {@link KeyValue} pair record the record's key is preserved. + *

      + * Use {@link ValueTransformerSupplier} to provide new instances of {@code ValueTransformer} to Kafka Stream's runtime. + *

      + * If a record's key and value should be modified {@link Transformer} can be used. * - * @param value type - * @param return type + * @param value type + * @param transformed value type + * @see ValueTransformerSupplier + * @see KStream#transformValues(ValueTransformerSupplier, String...) + * @see Transformer */ -public interface ValueTransformer { +@InterfaceStability.Unstable +public interface ValueTransformer { /** - * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology - * that contains it is initialized. + * Initialize this transformer. + * This is called once per instance when the topology gets initialized. *

      - * If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should - * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * The provided {@link ProcessorContext context} can be used to access topology and record meta data, to + * {@link ProcessorContext#schedule(long) schedule itself} for periodical calls (cf. {@link #punctuate(long)}), and + * to access attached {@link StateStore}s. + *

      + * Note that {@link ProcessorContext} is updated in the background with the current record's meta data. + * Thus, it only contains valid record meta data when accessed within {@link #transform(Object)}. + *

      + * Note that using {@link ProcessorContext#forward(Object, Object)}, + * {@link ProcessorContext#forward(Object, Object, int)}, or + * {@link ProcessorContext#forward(Object, Object, String)} is not allowed within any method of + * {@code ValueTransformer} and will result in an {@link StreamsException exception}. * - * @param context the context; may not be null + * @param context the context */ void init(final ProcessorContext context); /** - * Transform the record with the given key and value. + * Transform the given value to a new value. + * Additionally, any {@link StateStore} that is {@link KStream#transformValues(ValueTransformerSupplier, String...) + * attached} to this operator can be accessed and modified arbitrarily (cf. + * {@link ProcessorContext#getStateStore(String)}). + *

      + * Note, that using {@link ProcessorContext#forward(Object, Object)}, + * {@link ProcessorContext#forward(Object, Object, int)}, and + * {@link ProcessorContext#forward(Object, Object, String)} is not allowed within {@code transform} and + * will result in an {@link StreamsException exception}. * - * @param value the value for the record - * @return new value + * @param value the value to be transformed + * @return the new value */ - R transform(final V value); + VR transform(final V value); /** - * Perform any periodic operations and possibly return a new value, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context - * during {@link #init(ProcessorContext) initialization}. + * Perform any periodic operations if this processor {@link ProcessorContext#schedule(long) schedule itself} with + * the context during {@link #init(ProcessorContext) initialization}. + *

      + * It is not possible to return any new output records within {@code punctuate}. + * Using {@link ProcessorContext#forward(Object, Object)}, {@link ProcessorContext#forward(Object, Object, int)}, + * or {@link ProcessorContext#forward(Object, Object, String)} will result in an + * {@link StreamsException exception}. + * Furthermore, {@code punctuate} must return {@code null}. + *

      + * Note, that {@code punctuate} is called base on stream time (i.e., time progress with regard to + * timestamps return by the used {@link TimestampExtractor}) + * and not based on wall-clock time. * - * @param timestamp the stream time when this method is being called - * @return new value; if null it will not be forwarded to down stream + * @param timestamp the stream time when {@code punctuate} is being called + * @return must return {@code null}—otherwise, an {@link StreamsException exception} will be thrown */ - R punctuate(final long timestamp); + VR punctuate(final long timestamp); /** * Close this processor and clean up any resources. + *

      + * It is not possible to return any new output records within {@code close()}. + * Using {@link ProcessorContext#forward(Object, Object)}, {@link ProcessorContext#forward(Object, Object, int)}, + * or {@link ProcessorContext#forward(Object, Object, String)} will result in an {@link StreamsException exception}. */ void close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java index ecd454abac15d..01c920e95e4e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java @@ -14,18 +14,28 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; + /** - * A {@link ValueTransformerSupplier} interface which can create one or more {@link ValueTransformer} instances. + * A {@code ValueTransformerSupplier} interface which can create one or more {@link ValueTransformer} instances. + * + * @param value type + * @param transformed value type + * @see ValueTransformer + * @see KStream#transformValues(ValueTransformerSupplier, String...) + * @see Transformer + * @see TransformerSupplier + * @see KStream#transform(TransformerSupplier, String...) */ -public interface ValueTransformerSupplier { +@InterfaceStability.Unstable +public interface ValueTransformerSupplier { /** * Return a new {@link ValueTransformer} instance. * - * @return a new {@link ValueTransformer} instance. + * @return a new {@link ValueTransformer} instance. */ - ValueTransformer get(); + ValueTransformer get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 13a9529d6bedd..0c8b3047d5560 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -5,72 +5,84 @@ * 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 - * + *

      + * 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.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.processor.TimestampExtractor; + /** * A single window instance, defined by its start and end timestamp. + * {@code Window} is agnostic if start/end boundaries are inclusive or exclusive; this is defined by concrete + * window implementations. + *

      + * To specify how {@code Window} boundaries are defined use {@link Windows}. + * For time semantics, see {@link TimestampExtractor}. + * + * @see Windows + * @see org.apache.kafka.streams.kstream.internals.TimeWindow + * @see org.apache.kafka.streams.kstream.internals.SessionWindow + * @see org.apache.kafka.streams.kstream.internals.UnlimitedWindow + * @see TimestampExtractor */ +@InterfaceStability.Unstable public abstract class Window { - protected final long start; - protected final long end; + protected final long startMs; + protected final long endMs; /** - * Create a new window for the given start time (inclusive) and end time (exclusive). + * Create a new window for the given start and end time. * - * @param start the start timestamp of the window (inclusive) - * @param end the end timestamp of the window (exclusive) - * @throws IllegalArgumentException if {@code start} or {@code end} is negative or if {@code end} is smaller than - * {@code start} + * @param startMs the start timestamp of the window + * @param endMs the end timestamp of the window + * @throws IllegalArgumentException if {@code startMs} is negative or if {@code endMs} is smaller than {@code startMs} */ - public Window(long start, long end) throws IllegalArgumentException { - if (start < 0) { - throw new IllegalArgumentException("Window start time cannot be negative."); - } - if (end < 0) { - throw new IllegalArgumentException("Window end time cannot be negative."); + public Window(final long startMs, final long endMs) throws IllegalArgumentException { + if (startMs < 0) { + throw new IllegalArgumentException("Window startMs time cannot be negative."); } - if (end < start) { - throw new IllegalArgumentException("Window end time cannot be smaller than window start time."); + if (endMs < startMs) { + throw new IllegalArgumentException("Window endMs time cannot be smaller than window startMs time."); } - this.start = start; - this.end = end; + this.startMs = startMs; + this.endMs = endMs; } /** - * Return the start timestamp of this window, inclusive + * Return the start timestamp of this window. */ public long start() { - return start; + return startMs; } /** - * Return the end timestamp of this window, exclusive + * Return the end timestamp of this window. */ public long end() { - return end; + return endMs; } /** * Check if the given window overlaps with this window. + * Should throw an {@link IllegalArgumentException} if the {@code other} window has a different type than {@code + * this} window. * - * @param other another window - * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise + * @param other another window of the same type + * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise */ - public abstract boolean overlap(Window other); + public abstract boolean overlap(final Window other); @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (obj == this) { return true; } @@ -79,21 +91,20 @@ public boolean equals(Object obj) { return false; } - Window other = (Window) obj; - return this.start == other.start && this.end == other.end; + final Window other = (Window) obj; + return startMs == other.startMs && endMs == other.endMs; } @Override public int hashCode() { - long n = (this.start << 32) | this.end; - return (int) (n % 0xFFFFFFFFL); + return (int) (((startMs << 32) | endMs) % 0xFFFFFFFFL); } @Override public String toString() { return "Window{" + - "start=" + start + - ", end=" + end + - '}'; + "start=" + startMs + + ", end=" + endMs + + '}'; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java index 6606fcb367acc..63074c37b1593 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java @@ -17,20 +17,38 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; + /** - * Used to represent windowed stream aggregations (e.g. as returned by - * {@link KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, String)}), - * which have the type {@code , V>}. + * The result key type of a windowed stream aggregation. + *

      + * If a {@link KStream} gets grouped and aggregated using a window-aggregation the resulting {@link KTable} is a + * so-called "windowed {@link KTable}" with a combined key type that encodes the corresponding aggregation window and + * the original record key. + * Thus, a windowed {@link KTable} has type {@code ,V>}. * - * @param Type of the key + * @param type of the key + * @see KGroupedStream#count(Windows, String) + * @see KGroupedStream#count(Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#count(SessionWindows, String) + * @see KGroupedStream#count(SessionWindows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#reduce(Reducer, Windows, String) + * @see KGroupedStream#reduce(Reducer, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#reduce(Reducer, SessionWindows, String) + * @see KGroupedStream#reduce(Reducer, SessionWindows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, String) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Windows, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Merger, SessionWindows, org.apache.kafka.common.serialization.Serde, org.apache.kafka.streams.processor.StateStoreSupplier) + * @see KGroupedStream#aggregate(Initializer, Aggregator, Merger, SessionWindows, org.apache.kafka.common.serialization.Serde, String) */ +@InterfaceStability.Unstable public class Windowed { - private K key; + private final K key; - private Window window; + private final Window window; - public Windowed(K key, Window window) { + public Windowed(final K key, final Window window) { this.key = key; this.window = window; } @@ -47,7 +65,7 @@ public K key() { /** * Return the window containing the values associated with this key. * - * @return the window containing the values + * @return the window containing the values */ public Window window() { return window; @@ -59,21 +77,20 @@ public String toString() { } @Override - public boolean equals(Object obj) { + public boolean equals(final Object obj) { if (obj == this) return true; if (!(obj instanceof Windowed)) return false; - Windowed that = (Windowed) obj; - - return this.window.equals(that.window) && this.key.equals(that.key); + final Windowed that = (Windowed) obj; + return window.equals(that.window) && key.equals(that.key); } @Override public int hashCode() { - long n = ((long) window.hashCode() << 32) | key.hashCode(); + final long n = ((long) window.hashCode() << 32) | key.hashCode(); return (int) (n % 0xFFFFFFFFL); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index ebd92fe7346e5..74fff47f34a19 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -5,80 +5,108 @@ * 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 - * + *

      + * 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.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.streams.processor.TimestampExtractor; + import java.util.Map; /** - * The window specification interface that can be extended for windowing operation in joins and aggregations. + * The window specification interface for fixed size windows that is used to define window boundaries and window + * maintain duration. + *

      + * If not explicitly specified, the default maintain duration is 1 day. + * For time semantics, see {@link TimestampExtractor}. * - * @param type of the window instance + * @param type of the window instance + * @see TimeWindows + * @see UnlimitedWindows + * @see JoinWindows + * @see SessionWindows + * @see TimestampExtractor */ +@InterfaceStability.Unstable public abstract class Windows { private static final int DEFAULT_NUM_SEGMENTS = 3; - static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day + static final long DEFAULT_MAINTAIN_DURATION_MS = 24 * 60 * 60 * 1000L; // one day private long maintainDurationMs; public int segments; protected Windows() { - this.segments = DEFAULT_NUM_SEGMENTS; - this.maintainDurationMs = DEFAULT_MAINTAIN_DURATION; + segments = DEFAULT_NUM_SEGMENTS; + maintainDurationMs = DEFAULT_MAINTAIN_DURATION_MS; } /** - * Set the window maintain duration in milliseconds of streams time. + * Set the window maintain duration (retention time) in milliseconds. * This retention time is a guaranteed lower bound for how long a window will be maintained. * - * @return itself + * @param durationMs the window retention time in milliseconds + * @return itself + * @throws IllegalArgumentException if {@code durationMs} is negative */ - public Windows until(long durationMs) { - this.maintainDurationMs = durationMs; + // This should always get overridden to provide the correct return type and thus to avoid a cast + public Windows until(final long durationMs) throws IllegalArgumentException { + if (durationMs < 0) { + throw new IllegalArgumentException("Window retention time (durationMs) cannot be negative."); + } + maintainDurationMs = durationMs; return this; } /** - * Specify the number of segments to be used for rolling the window store, - * this function is not exposed to users but can be called by developers that extend this JoinWindows specs. + * Return the window maintain duration (retention time) in milliseconds. * - * @return itself + * @return the window maintain duration */ - protected Windows segments(int segments) { + public long maintainMs() { + return maintainDurationMs; + } + + /** + * Set the number of segments to be used for rolling the window store. + * This function is not exposed to users but can be called by developers that extend this class. + * + * @param segments the number of segments to be used + * @return itself + * @throws IllegalArgumentException if specified segments is small than 2 + */ + protected Windows segments(final int segments) throws IllegalArgumentException { + if (segments < 2) { + throw new IllegalArgumentException("Number of segments must be at least 2."); + } this.segments = segments; return this; } /** - * Return the window maintain duration in milliseconds of streams time. + * Create all windows that contain the provided timestamp, indexed by non-negative window start timestamps. * - * @return the window maintain duration in milliseconds of streams time + * @param timestamp the timestamp window should get created for + * @return a map of {@code windowStartTimestamp -> Window} entries */ - public long maintainMs() { - return this.maintainDurationMs; - } + public abstract Map windowsFor(final long timestamp); /** - * Creates all windows that contain the provided timestamp, indexed by non-negative window start timestamps. + * Return the size of the specified windows in milliseconds. * - * @param timestamp the timestamp window should get created for - * @return a map of {@code windowStartTimestamp -> Window} entries + * @return the size of the specified windows */ - public abstract Map windowsFor(long timestamp); - public abstract long size(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index 31a3dc63b3416..bcffce2dcb227 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -37,12 +37,6 @@ public abstract class AbstractStream { protected final String name; protected final Set sourceNodes; - public AbstractStream(AbstractStream stream) { - this.topology = stream.topology; - this.name = stream.name; - this.sourceNodes = stream.sourceNodes; - } - public AbstractStream(KStreamBuilder topology, String name, Set sourceNodes) { if (sourceNodes == null || sourceNodes.isEmpty()) { throw new IllegalArgumentException("parameter must not be null or empty"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index b99e55d29d8e6..7b9fd946805dc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -349,23 +349,23 @@ public void to(Serde keySerde, Serde valSerde, String topic) { @SuppressWarnings("unchecked") @Override - public void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic) { + public void to(final Serde keySerde, final Serde valSerde, StreamPartitioner partitioner, final String topic) { Objects.requireNonNull(topic, "topic can't be null"); - String name = topology.newName(SINK_NAME); + final String name = topology.newName(SINK_NAME); - Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); - Serializer valSerializer = valSerde == null ? null : valSerde.serializer(); + final Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); + final Serializer valSerializer = valSerde == null ? null : valSerde.serializer(); if (partitioner == null && keySerializer != null && keySerializer instanceof WindowedSerializer) { - WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; - partitioner = (StreamPartitioner) new WindowedStreamPartitioner(windowedSerializer); + final WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; + partitioner = (StreamPartitioner) new WindowedStreamPartitioner(topic, windowedSerializer); } topology.addSink(name, topic, keySerializer, valSerializer, partitioner, this.name); } @Override - public KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames) { + public KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames) { Objects.requireNonNull(transformerSupplier, "transformerSupplier can't be null"); String name = topology.newName(TRANSFORM_NAME); @@ -711,20 +711,20 @@ public KStream join(KStream lhs, KStreamJoinWindow thisWindowedStream = new KStreamJoinWindow<>(thisWindow.name(), - windows.before + windows.after + 1, + windows.beforeMs + windows.afterMs + 1, windows.maintainMs()); KStreamJoinWindow otherWindowedStream = new KStreamJoinWindow<>(otherWindow.name(), - windows.before + windows.after + 1, + windows.beforeMs + windows.afterMs + 1, windows.maintainMs()); final KStreamKStreamJoin joinThis = new KStreamKStreamJoin<>(otherWindow.name(), - windows.before, - windows.after, + windows.beforeMs, + windows.afterMs, joiner, leftOuter); final KStreamKStreamJoin joinOther = new KStreamKStreamJoin<>(thisWindow.name(), - windows.after, - windows.before, + windows.afterMs, + windows.beforeMs, reverseJoiner(joiner), rightOuter); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index f689c9503fcdd..acc013f8ff35b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -28,8 +28,6 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.File; import java.util.Map; @@ -49,7 +47,6 @@ public Processor get() { public static class KStreamTransformValuesProcessor implements Processor { - private static final Logger log = LoggerFactory.getLogger(KStreamTransformValuesProcessor.class); private final ValueTransformer valueTransformer; private ProcessorContext context; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index 21a11766a4f99..f8e3f15791ae2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; @@ -76,14 +75,12 @@ public void init(ProcessorContext context) { valueGetter.init(context); } - /** - * @throws StreamsException if key is null - */ @Override public void process(final K key, final Change change) { - // the keys should never be null - if (key == null) - throw new StreamsException("Record key for KTable join operator should not be null."); + // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record + if (key == null) { + return; + } R newValue = null; R oldValue = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index cfdaf6123a7ac..ea625a7880e46 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -66,14 +65,12 @@ public void init(ProcessorContext context) { valueGetter.init(context); } - /** - * @throws StreamsException if key is null - */ @Override public void process(final K key, final Change change) { - // the keys should never be null - if (key == null) - throw new StreamsException("Record key for KTable left-join operator should not be null."); + // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record + if (key == null) { + return; + } R newValue = null; R oldValue = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 5b3241536ff5a..0809d65b58506 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -65,14 +64,12 @@ public void init(ProcessorContext context) { valueGetter.init(context); } - /** - * @throws StreamsException if key is null - */ @Override public void process(final K key, final Change change) { - // the keys should never be null - if (key == null) - throw new StreamsException("Record key for KTable outer-join operator should not be null."); + // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record + if (key == null) { + return; + } R newValue = null; R oldValue = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 17fa5cb2d2db5..938bd436ac838 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -66,14 +65,12 @@ public void init(ProcessorContext context) { valueGetter.init(context); } - /** - * @throws StreamsException if key is null - */ @Override public void process(final K key, final Change change) { - // the keys should never be null - if (key == null) - throw new StreamsException("Record key for KTable right-join operator should not be null."); + // we do join iff keys are equal, thus, if key is null we cannot join and just ignore the record + if (key == null) { + return; + } final R newValue; R oldValue = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index d3911515d7c08..93d9de7eb576b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; @@ -58,9 +57,9 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { - // the keys should never be null + // if the key is null, then ignore the record if (key == null) - throw new StreamsException("Record key for the source KTable from store name " + storeName + " should not be null."); + return; V oldValue = store.get(key); store.put(key, value); tupleForwarder.maybeForward(key, value, oldValue); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java index 48213d67da153..69097730e6349 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionKeySerde.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.kstream.Window; import org.apache.kafka.streams.kstream.Windowed; import java.nio.ByteBuffer; @@ -32,7 +33,6 @@ */ public class SessionKeySerde implements Serde> { private static final int TIMESTAMP_SIZE = 8; - private static final String SESSIONKEY = "sessionkey"; private final Serde keySerde; @@ -76,7 +76,7 @@ public byte[] serialize(final String topic, final Windowed data) { if (data == null) { return null; } - return toBinary(data, keySerializer).get(); + return toBinary(data, keySerializer, topic).get(); } @Override @@ -101,7 +101,7 @@ public Windowed deserialize(final String topic, final byte[] data) { if (data == null || data.length == 0) { return null; } - return from(data, deserializer); + return from(data, deserializer, topic); } @@ -126,24 +126,49 @@ public static byte[] extractKeyBytes(final byte[] binaryKey) { return bytes; } - public static Windowed from(final byte[] binaryKey, final Deserializer keyDeserializer) { - final K key = extractKey(binaryKey, keyDeserializer); + public static Windowed from(final byte[] binaryKey, final Deserializer keyDeserializer, final String topic) { + final K key = extractKey(binaryKey, keyDeserializer, topic); final ByteBuffer buffer = ByteBuffer.wrap(binaryKey); final long start = buffer.getLong(binaryKey.length - TIMESTAMP_SIZE); final long end = buffer.getLong(binaryKey.length - 2 * TIMESTAMP_SIZE); return new Windowed<>(key, new SessionWindow(start, end)); } - private static K extractKey(final byte[] binaryKey, Deserializer deserializer) { - return deserializer.deserialize(SESSIONKEY, extractKeyBytes(binaryKey)); + private static K extractKey(final byte[] binaryKey, Deserializer deserializer, final String topic) { + return deserializer.deserialize(topic, extractKeyBytes(binaryKey)); } - public static Bytes toBinary(final Windowed sessionKey, final Serializer serializer) { - final byte[] bytes = serializer.serialize(SESSIONKEY, sessionKey.key()); + public static Bytes toBinary(final Windowed sessionKey, final Serializer serializer, final String topic) { + final byte[] bytes = serializer.serialize(topic, sessionKey.key()); ByteBuffer buf = ByteBuffer.allocate(bytes.length + 2 * TIMESTAMP_SIZE); buf.put(bytes); buf.putLong(sessionKey.window().end()); buf.putLong(sessionKey.window().start()); return new Bytes(buf.array()); } + + public static Bytes bytesToBinary(final Windowed sessionKey) { + final byte[] bytes = sessionKey.key().get(); + ByteBuffer buf = ByteBuffer.allocate(bytes.length + 2 * TIMESTAMP_SIZE); + buf.put(bytes); + buf.putLong(sessionKey.window().end()); + buf.putLong(sessionKey.window().start()); + return new Bytes(buf.array()); + } + + public static Window extractWindow(final byte [] binaryKey) { + final ByteBuffer buffer = ByteBuffer.wrap(binaryKey); + final long start = buffer.getLong(binaryKey.length - TIMESTAMP_SIZE); + final long end = buffer.getLong(binaryKey.length - 2 * TIMESTAMP_SIZE); + return new TimeWindow(start, end); + } + + public static Windowed fromBytes(Bytes bytesKey) { + final byte[] binaryKey = bytesKey.get(); + final ByteBuffer buffer = ByteBuffer.wrap(binaryKey); + final long start = buffer.getLong(binaryKey.length - TIMESTAMP_SIZE); + final long end = buffer.getLong(binaryKey.length - 2 * TIMESTAMP_SIZE); + return new Windowed<>(Bytes.wrap(extractKeyBytes(binaryKey)), new SessionWindow(start, end)); + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java index db63029e04ae7..2ea273c0bcb3e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/SessionWindow.java @@ -18,41 +18,47 @@ */ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.kstream.Window; /** * A session window covers a closed time interval with its start and end timestamp both being an inclusive boundary. + *

      + * For time semantics, see {@link org.apache.kafka.streams.processor.TimestampExtractor TimestampExtractor}. * * @see TimeWindow * @see UnlimitedWindow * @see org.apache.kafka.streams.kstream.SessionWindows + * @see org.apache.kafka.streams.processor.TimestampExtractor */ +@InterfaceStability.Unstable public final class SessionWindow extends Window { /** * Create a new window for the given start time and end time (both inclusive). * - * @param start the start timestamp of the window - * @param end the end timestamp of the window + * @param startMs the start timestamp of the window + * @param endMs the end timestamp of the window + * @throws IllegalArgumentException if {@code startMs} is negative or if {@code endMs} is smaller than {@code startMs} */ - public SessionWindow(final long start, final long end) { - super(start, end); + public SessionWindow(final long startMs, final long endMs) throws IllegalArgumentException { + super(startMs, endMs); } /** * Check if the given window overlaps with this window. * - * @param other another window - * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise + * @param other another window + * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise * @throws IllegalArgumentException if the {@code other} window has a different type than {@link this} window */ public boolean overlap(final Window other) throws IllegalArgumentException { if (getClass() != other.getClass()) { throw new IllegalArgumentException("Cannot compare windows of different type. Other window has type " - + other.getClass()); + + other.getClass() + "."); } final SessionWindow otherWindow = (SessionWindow) other; - return !(otherWindow.end < start || end < otherWindow.start); + return !(otherWindow.endMs < startMs || endMs < otherWindow.startMs); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java index 630821fdb8064..ab805acaa2200 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java @@ -14,25 +14,57 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.kstream.Window; +/** + * A {@link TimeWindow} covers a half-open time interval with its start timestamp as an inclusive boundary and its end + * timestamp as exclusive boundary. + * It is a fixed size window, i.e., all instances (of a single {@link org.apache.kafka.streams.kstream.TimeWindows + * window specification}) will have the same size. + *

      + * For time semantics, see {@link org.apache.kafka.streams.processor.TimestampExtractor TimestampExtractor}. + * + * @see SessionWindow + * @see UnlimitedWindow + * @see org.apache.kafka.streams.kstream.TimeWindows + * @see org.apache.kafka.streams.processor.TimestampExtractor + */ +@InterfaceStability.Unstable public class TimeWindow extends Window { - public TimeWindow(long start, long end) { - super(start, end); + /** + * Create a new window for the given start time (inclusive) and end time (exclusive). + * + * @param startMs the start timestamp of the window (inclusive) + * @param endMs the end timestamp of the window (exclusive) + * @throws IllegalArgumentException if {@code startMs} is negative or if {@code endMs} is smaller than or equal to + * {@code startMs} + */ + public TimeWindow(final long startMs, final long endMs) throws IllegalArgumentException { + super(startMs, endMs); + if (startMs == endMs) { + throw new IllegalArgumentException("Window endMs must be greater than window startMs."); + } } + /** + * Check if the given window overlaps with this window. + * + * @param other another window + * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise + * @throws IllegalArgumentException if the {@code other} window has a different type than {@link this} window + */ @Override - public boolean overlap(Window other) { + public boolean overlap(final Window other) throws IllegalArgumentException { if (getClass() != other.getClass()) { throw new IllegalArgumentException("Cannot compare windows of different type. Other window has type " - + other.getClass()); + + other.getClass() + "."); } final TimeWindow otherWindow = (TimeWindow) other; - return start < otherWindow.end && otherWindow.start < end; + return startMs < otherWindow.endMs && otherWindow.startMs < endMs; } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java index e9ec040d0d191..311169e58e0be 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java @@ -5,33 +5,61 @@ * 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 - * + *

      + * 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.kstream.internals; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.kstream.Window; +/** + * {@link UnlimitedWindow} is an "infinite" large window with a fixed (inclusive) start time. + * All windows of the same {@link org.apache.kafka.streams.kstream.UnlimitedWindows window specification} will have the + * same start time. + * To make the window size "infinite" end time is set to {@link Long#MAX_VALUE}. + *

      + * For time semantics, see {@link org.apache.kafka.streams.processor.TimestampExtractor TimestampExtractor}. + * + * @see TimeWindow + * @see SessionWindow + * @see org.apache.kafka.streams.kstream.UnlimitedWindows + * @see org.apache.kafka.streams.processor.TimestampExtractor + */ +@InterfaceStability.Unstable public class UnlimitedWindow extends Window { - public UnlimitedWindow(long start) { - super(start, Long.MAX_VALUE); + /** + * Create a new window for the given start time (inclusive). + * + * @param startMs the start timestamp of the window (inclusive) + * @throws IllegalArgumentException if {@code start} is negative + */ + public UnlimitedWindow(final long startMs) { + super(startMs, Long.MAX_VALUE); } + /** + * Returns {@code true} if the given window is of the same type, because all unlimited windows overlap with each + * other due to their infinite size. + * + * @param other another window + * @return {@code true} + * @throws IllegalArgumentException if the {@code other} window has a different type than {@link this} window + */ @Override - public boolean overlap(Window other) { + public boolean overlap(final Window other) { if (getClass() != other.getClass()) { throw new IllegalArgumentException("Cannot compare windows of different type. Other window has type " - + other.getClass()); + + other.getClass() + "."); } return true; } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java index ba9873b436371..ad54fcb479d72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java @@ -24,9 +24,11 @@ public class WindowedStreamPartitioner implements StreamPartitioner, V> { + private final String topic; private final WindowedSerializer serializer; - public WindowedStreamPartitioner(WindowedSerializer serializer) { + WindowedStreamPartitioner(final String topic, final WindowedSerializer serializer) { + this.topic = topic; this.serializer = serializer; } @@ -40,8 +42,8 @@ public WindowedStreamPartitioner(WindowedSerializer serializer) { * @param numPartitions the total number of partitions * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used */ - public Integer partition(Windowed windowedKey, V value, int numPartitions) { - byte[] keyBytes = serializer.serializeBaseKey(null, windowedKey); + public Integer partition(final Windowed windowedKey, final V value, final int numPartitions) { + final byte[] keyBytes = serializer.serializeBaseKey(topic, windowedKey); // hash the keyBytes to choose a partition return toPositive(Utils.murmur2(keyBytes)) % numPartitions; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 1da12097b53eb..25efcd6eaf391 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -83,6 +84,7 @@ protected int maxNumPartitions(Cluster metadata, Set topics) { if (partitions == null) { log.info("Skipping assigning topic {} to tasks since its metadata is not available yet", topic); + return StreamPartitionAssignor.NOT_AVAILABLE; } else { int numPartitions = partitions.size(); if (numPartitions > maxNumPartitions) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index beaace349e085..e0eca8c90befe 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -55,6 +55,8 @@ public interface Processor { /** * Close this processor and clean up any resources. Be aware that {@link #close()} is called after an internal cleanup. * Thus, it is not possible to write anything to Kafka as underlying clients are already closed. + *

      + * Note: Do not close any streams managed resources, like {@link StateStore}s here, as they are managed by the library. */ void close(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java index 9aa0932dc656e..f417a346ed763 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java @@ -49,6 +49,9 @@ public interface StateStore { * Close the storage engine. * Note that this function needs to be idempotent since it may be called * several times on the same state store. + *

      + * Users only need to implement this function but should NEVER need to call this api explicitly + * as it will be called by the library automatically when necessary */ void close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index b25fcad3d1f2f..92cfda7a00777 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -49,6 +49,7 @@ import java.util.Set; import java.util.regex.Pattern; + /** * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors, * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to @@ -81,7 +82,7 @@ public class TopologyBuilder { private final List> copartitionSourceGroups = new ArrayList<>(); // map from source processor names to subscribed topics (without application-id prefix for internal topics) - private final HashMap nodeToSourceTopics = new HashMap<>(); + private final HashMap> nodeToSourceTopics = new HashMap<>(); // map from source processor names to regex subscription patterns private final HashMap nodeToSourcePatterns = new LinkedHashMap<>(); @@ -97,6 +98,10 @@ public class TopologyBuilder { // are connected to these state stores private final Map> stateStoreNameToSourceTopics = new HashMap<>(); + // map from state store names to all the regex subscribed topics from source processors that + // are connected to these state stores + private final Map> stateStoreNameToSourceRegex = new HashMap<>(); + // map from state store names to this state store's corresponding changelog topic if possible, // this is used in the extended KStreamBuilder. private final Map storeToChangelogTopic = new HashMap<>(); @@ -146,11 +151,11 @@ private static abstract class NodeFactory { } private static class ProcessorNodeFactory extends NodeFactory { - public final String[] parents; - private final ProcessorSupplier supplier; + private final String[] parents; + private final ProcessorSupplier supplier; private final Set stateStoreNames = new HashSet<>(); - public ProcessorNodeFactory(String name, String[] parents, ProcessorSupplier supplier) { + ProcessorNodeFactory(String name, String[] parents, ProcessorSupplier supplier) { super(name); this.parents = parents.clone(); this.supplier = supplier; @@ -160,37 +165,32 @@ public void addStateStore(String stateStoreName) { stateStoreNames.add(stateStoreName); } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { - return new ProcessorNode(name, supplier.get(), stateStoreNames); + return new ProcessorNode<>(name, supplier.get(), stateStoreNames); } } private class SourceNodeFactory extends NodeFactory { - private final String[] topics; - public final Pattern pattern; - private Deserializer keyDeserializer; - private Deserializer valDeserializer; + private final List topics; + private final Pattern pattern; + private final Deserializer keyDeserializer; + private final Deserializer valDeserializer; - private SourceNodeFactory(String name, String[] topics, Pattern pattern, Deserializer keyDeserializer, Deserializer valDeserializer) { + private SourceNodeFactory(String name, String[] topics, Pattern pattern, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); - this.topics = topics != null ? topics.clone() : null; + this.topics = topics != null ? Arrays.asList(topics) : new ArrayList(); this.pattern = pattern; this.keyDeserializer = keyDeserializer; this.valDeserializer = valDeserializer; } - String[] getTopics() { - return topics; - } - - String[] getTopics(Collection subscribedTopics) { + List getTopics(Collection subscribedTopics) { // if it is subscribed via patterns, it is possible that the topic metadata has not been updated // yet and hence the map from source node to topics is stale, in this case we put the pattern as a place holder; // this should only happen for debugging since during runtime this function should always be called after the metadata has updated. if (subscribedTopics.isEmpty()) - return new String[] {"Pattern[" + pattern + "]"}; + return Collections.singletonList("Pattern[" + pattern + "]"); List matchedTopics = new ArrayList<>(); for (String update : subscribedTopics) { @@ -207,21 +207,20 @@ String[] getTopics(Collection subscribedTopics) { matchedTopics.add(update); } } - return matchedTopics.toArray(new String[matchedTopics.size()]); + return matchedTopics; } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { - final String[] sourceTopics = nodeToSourceTopics.get(name); + final List sourceTopics = nodeToSourceTopics.get(name); // if it is subscribed via patterns, it is possible that the topic metadata has not been updated // yet and hence the map from source node to topics is stale, in this case we put the pattern as a place holder; // this should only happen for debugging since during runtime this function should always be called after the metadata has updated. if (sourceTopics == null) - return new SourceNode(name, new String[] {"Pattern[" + pattern + "]"}, keyDeserializer, valDeserializer); + return new SourceNode<>(name, Collections.singletonList("Pattern[" + pattern + "]"), keyDeserializer, valDeserializer); else - return new SourceNode(name, maybeDecorateInternalSourceTopics(sourceTopics).toArray(new String[sourceTopics.length]), keyDeserializer, valDeserializer); + return new SourceNode<>(name, maybeDecorateInternalSourceTopics(sourceTopics), keyDeserializer, valDeserializer); } private boolean isMatch(String topic) { @@ -229,14 +228,14 @@ private boolean isMatch(String topic) { } } - private class SinkNodeFactory extends NodeFactory { - public final String[] parents; - public final String topic; - private Serializer keySerializer; - private Serializer valSerializer; - private final StreamPartitioner partitioner; + private class SinkNodeFactory extends NodeFactory { + private final String[] parents; + private final String topic; + private final Serializer keySerializer; + private final Serializer valSerializer; + private final StreamPartitioner partitioner; - private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { + private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { super(name); this.parents = parents.clone(); this.topic = topic; @@ -245,14 +244,13 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer this.partitioner = partitioner; } - @SuppressWarnings("unchecked") @Override public ProcessorNode build() { if (internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id - return new SinkNode(name, decorateTopic(topic), keySerializer, valSerializer, partitioner); + return new SinkNode<>(name, decorateTopic(topic), keySerializer, valSerializer, partitioner); } else { - return new SinkNode(name, topic, keySerializer, valSerializer, partitioner); + return new SinkNode<>(name, topic, keySerializer, valSerializer, partitioner); } } } @@ -263,7 +261,7 @@ public static class TopicsInfo { public Map stateChangelogTopics; public Map repartitionSourceTopics; - public TopicsInfo(Set sinkTopics, Set sourceTopics, Map repartitionSourceTopics, Map stateChangelogTopics) { + TopicsInfo(Set sinkTopics, Set sourceTopics, Map repartitionSourceTopics, Map stateChangelogTopics) { this.sinkTopics = sinkTopics; this.sourceTopics = sourceTopics; this.stateChangelogTopics = stateChangelogTopics; @@ -312,13 +310,13 @@ public TopologyBuilder() {} /** * Set the applicationId to be used for auto-generated internal topics. * - * This is required before calling {@link #sourceTopics}, {@link #topicGroups}, - * {@link #copartitionSources}, {@link #stateStoreNameToSourceTopics} and {@link #build(Integer)}. + * This is required before calling {@link #topicGroups}, {@link #copartitionSources}, + * {@link #stateStoreNameToSourceTopics} and {@link #build(Integer)}. * * @param applicationId the streams applicationId. Should be the same as set by * {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_ID_CONFIG} */ - public synchronized final TopologyBuilder setApplicationId(String applicationId) { + public synchronized final TopologyBuilder setApplicationId(final String applicationId) { Objects.requireNonNull(applicationId, "applicationId can't be null"); this.applicationId = applicationId; @@ -336,8 +334,8 @@ public synchronized final TopologyBuilder setApplicationId(String applicationId) * @param topics the name of one or more Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder addSource(String name, String... topics) { - return addSource(null, name, (Deserializer) null, (Deserializer) null, topics); + public synchronized final TopologyBuilder addSource(final String name, final String... topics) { + return addSource(null, name, null, null, topics); } /** @@ -352,8 +350,8 @@ public synchronized final TopologyBuilder addSource(String name, String... topic * @param topics the name of one or more Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, String... topics) { - return addSource(offsetReset, name, (Deserializer) null, (Deserializer) null, topics); + public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, final String name, final String... topics) { + return addSource(offsetReset, name, null, null, topics); } @@ -369,8 +367,8 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @param topicPattern regular expression pattern to match Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder addSource(String name, Pattern topicPattern) { - return addSource(null, name, (Deserializer) null, (Deserializer) null, topicPattern); + public synchronized final TopologyBuilder addSource(final String name, final Pattern topicPattern) { + return addSource(null, name, null, null, topicPattern); } /** @@ -386,8 +384,8 @@ public synchronized final TopologyBuilder addSource(String name, Pattern topicPa * @param topicPattern regular expression pattern to match Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, Pattern topicPattern) { - return addSource(offsetReset, name, (Deserializer) null, (Deserializer) null, topicPattern); + public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, final String name, final Pattern topicPattern) { + return addSource(offsetReset, name, null, null, topicPattern); } @@ -407,7 +405,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if processor is already added or if topics have already been registered by another source */ - public synchronized final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public synchronized final TopologyBuilder addSource(final String name, final Deserializer keyDeserializer, final Deserializer valDeserializer, final String... topics) { return addSource(null, name, keyDeserializer, valDeserializer, topics); } @@ -429,7 +427,7 @@ public synchronized final TopologyBuilder addSource(String name, Deserializer ke * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if processor is already added or if topics have already been registered by another source */ - public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, final String name, final Deserializer keyDeserializer, final Deserializer valDeserializer, final String... topics) { if (topics.length == 0) { throw new TopologyBuilderException("You must provide at least one topic"); } @@ -445,7 +443,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, } nodeFactories.put(name, new SourceNodeFactory(name, topics, null, keyDeserializer, valDeserializer)); - nodeToSourceTopics.put(name, topics.clone()); + nodeToSourceTopics.put(name, Arrays.asList(topics)); nodeGrouper.add(name); return this; @@ -470,7 +468,7 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @param topic the topic to source the data from * @param processorName the name of the {@link ProcessorSupplier} * @param stateUpdateSupplier the instance of {@link ProcessorSupplier} - * @return + * @return this builder instance so methods can be chained together; never null */ public synchronized TopologyBuilder addGlobalStore(final StateStore store, final String sourceName, @@ -499,7 +497,7 @@ public synchronized TopologyBuilder addGlobalStore(final StateStore store, globalTopics.add(topic); final String[] topics = {topic}; nodeFactories.put(sourceName, new SourceNodeFactory(sourceName, topics, null, keyDeserializer, valueDeserializer)); - nodeToSourceTopics.put(sourceName, topics.clone()); + nodeToSourceTopics.put(sourceName, Arrays.asList(topics)); nodeGrouper.add(sourceName); final String[] parents = {sourceName}; @@ -547,7 +545,7 @@ private void validateTopicNotAlreadyRegistered(final String topic) { * @throws TopologyBuilderException if processor is already added or if topics have already been registered by name */ - public synchronized final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, Pattern topicPattern) { + public synchronized final TopologyBuilder addSource(final String name, final Deserializer keyDeserializer, final Deserializer valDeserializer, final Pattern topicPattern) { return addSource(null, name, keyDeserializer, valDeserializer, topicPattern); } @@ -573,7 +571,7 @@ public synchronized final TopologyBuilder addSource(String name, Deserializer ke * @throws TopologyBuilderException if processor is already added or if topics have already been registered by name */ - public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, String name, Deserializer keyDeserializer, Deserializer valDeserializer, Pattern topicPattern) { + public synchronized final TopologyBuilder addSource(final AutoOffsetReset offsetReset, final String name, final Deserializer keyDeserializer, final Deserializer valDeserializer, final Pattern topicPattern) { Objects.requireNonNull(topicPattern, "topicPattern can't be null"); Objects.requireNonNull(name, "name can't be null"); @@ -611,8 +609,8 @@ public synchronized final TopologyBuilder addSource(AutoOffsetReset offsetReset, * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ - public synchronized final TopologyBuilder addSink(String name, String topic, String... parentNames) { - return addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + public synchronized final TopologyBuilder addSink(final String name, final String topic, final String... parentNames) { + return addSink(name, topic, null, null, parentNames); } /** @@ -638,8 +636,8 @@ public synchronized final TopologyBuilder addSink(String name, String topic, Str * @see #addSink(String, String, Serializer, Serializer, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ - public synchronized final TopologyBuilder addSink(String name, String topic, StreamPartitioner partitioner, String... parentNames) { - return addSink(name, topic, (Serializer) null, (Serializer) null, partitioner, parentNames); + public synchronized final TopologyBuilder addSink(final String name, final String topic, final StreamPartitioner partitioner, final String... parentNames) { + return addSink(name, topic, null, null, partitioner, parentNames); } /** @@ -661,8 +659,8 @@ public synchronized final TopologyBuilder addSink(String name, String topic, Str * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, StreamPartitioner, String...) */ - public synchronized final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { - return addSink(name, topic, keySerializer, valSerializer, (StreamPartitioner) null, parentNames); + public synchronized final TopologyBuilder addSink(final String name, final String topic, final Serializer keySerializer, final Serializer valSerializer, final String... parentNames) { + return addSink(name, topic, keySerializer, valSerializer, null, parentNames); } /** @@ -686,7 +684,7 @@ public synchronized final TopologyBuilder addSink(String name, String topic, Ser * @see #addSink(String, String, Serializer, Serializer, String...) * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name */ - public synchronized final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner, String... parentNames) { + public synchronized final TopologyBuilder addSink(final String name, final String topic, final Serializer keySerializer, final Serializer valSerializer, final StreamPartitioner partitioner, final String... parentNames) { Objects.requireNonNull(name, "name must not be null"); Objects.requireNonNull(topic, "topic must not be null"); if (nodeFactories.containsKey(name)) @@ -703,7 +701,7 @@ public synchronized final TopologyBuilder addSink(String name, String top } } - nodeFactories.put(name, new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer, partitioner)); + nodeFactories.put(name, new SinkNodeFactory<>(name, parentNames, topic, keySerializer, valSerializer, partitioner)); nodeToSinkTopic.put(name, topic); nodeGrouper.add(name); nodeGrouper.unite(name, parentNames); @@ -720,7 +718,7 @@ public synchronized final TopologyBuilder addSink(String name, String top * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name */ - public synchronized final TopologyBuilder addProcessor(String name, ProcessorSupplier supplier, String... parentNames) { + public synchronized final TopologyBuilder addProcessor(final String name, final ProcessorSupplier supplier, final String... parentNames) { Objects.requireNonNull(name, "name must not be null"); Objects.requireNonNull(supplier, "supplier must not be null"); if (nodeFactories.containsKey(name)) @@ -749,7 +747,7 @@ public synchronized final TopologyBuilder addProcessor(String name, ProcessorSup * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if state store supplier is already added */ - public synchronized final TopologyBuilder addStateStore(StateStoreSupplier supplier, String... processorNames) { + public synchronized final TopologyBuilder addStateStore(final StateStoreSupplier supplier, final String... processorNames) { Objects.requireNonNull(supplier, "supplier can't be null"); if (stateFactories.containsKey(supplier.name())) { throw new TopologyBuilderException("StateStore " + supplier.name() + " is already added."); @@ -773,7 +771,7 @@ public synchronized final TopologyBuilder addStateStore(StateStoreSupplier suppl * @param stateStoreNames the names of state stores that the processor uses * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder connectProcessorAndStateStores(String processorName, String... stateStoreNames) { + public synchronized final TopologyBuilder connectProcessorAndStateStores(final String processorName, final String... stateStoreNames) { Objects.requireNonNull(processorName, "processorName can't be null"); if (stateStoreNames != null) { for (String stateStoreName : stateStoreNames) { @@ -788,7 +786,7 @@ public synchronized final TopologyBuilder connectProcessorAndStateStores(String * This is used only for KStreamBuilder: when adding a KTable from a source topic, * we need to add the topic as the KTable's materialized state store's changelog. */ - protected synchronized final TopologyBuilder connectSourceStoreAndTopic(String sourceStoreName, String topic) { + protected synchronized final TopologyBuilder connectSourceStoreAndTopic(final String sourceStoreName, final String topic) { if (storeToChangelogTopic.containsKey(sourceStoreName)) { throw new TopologyBuilderException("Source store " + sourceStoreName + " is already added."); } @@ -806,7 +804,7 @@ protected synchronized final TopologyBuilder connectSourceStoreAndTopic(String s * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if less than two processors are specified, or if one of the processors is not added yet */ - public synchronized final TopologyBuilder connectProcessors(String... processorNames) { + public synchronized final TopologyBuilder connectProcessors(final String... processorNames) { if (processorNames.length < 2) throw new TopologyBuilderException("At least two processors need to participate in the connection."); @@ -829,7 +827,7 @@ public synchronized final TopologyBuilder connectProcessors(String... processorN * @param topicName the name of the topic * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder addInternalTopic(String topicName) { + public synchronized final TopologyBuilder addInternalTopic(final String topicName) { Objects.requireNonNull(topicName, "topicName can't be null"); this.internalTopicNames.add(topicName); @@ -842,69 +840,85 @@ public synchronized final TopologyBuilder addInternalTopic(String topicName) { * @param sourceNodes a set of source node names * @return this builder instance so methods can be chained together; never null */ - public synchronized final TopologyBuilder copartitionSources(Collection sourceNodes) { + public synchronized final TopologyBuilder copartitionSources(final Collection sourceNodes) { copartitionSourceGroups.add(Collections.unmodifiableSet(new HashSet<>(sourceNodes))); return this; } - private void connectProcessorAndStateStore(String processorName, String stateStoreName) { + private void connectProcessorAndStateStore(final String processorName, final String stateStoreName) { if (!stateFactories.containsKey(stateStoreName)) throw new TopologyBuilderException("StateStore " + stateStoreName + " is not added yet."); if (!nodeFactories.containsKey(processorName)) throw new TopologyBuilderException("Processor " + processorName + " is not added yet."); - StateStoreFactory stateStoreFactory = stateFactories.get(stateStoreName); - Iterator iter = stateStoreFactory.users.iterator(); + final StateStoreFactory stateStoreFactory = stateFactories.get(stateStoreName); + final Iterator iter = stateStoreFactory.users.iterator(); if (iter.hasNext()) { - String user = iter.next(); + final String user = iter.next(); nodeGrouper.unite(user, processorName); } stateStoreFactory.users.add(processorName); NodeFactory nodeFactory = nodeFactories.get(processorName); if (nodeFactory instanceof ProcessorNodeFactory) { - ProcessorNodeFactory processorNodeFactory = (ProcessorNodeFactory) nodeFactory; + final ProcessorNodeFactory processorNodeFactory = (ProcessorNodeFactory) nodeFactory; processorNodeFactory.addStateStore(stateStoreName); - connectStateStoreNameToSourceTopics(stateStoreName, processorNodeFactory); + connectStateStoreNameToSourceTopicsOrPattern(stateStoreName, processorNodeFactory); } else { throw new TopologyBuilderException("cannot connect a state store " + stateStoreName + " to a source node or a sink node."); } } - private Set findSourceTopicsForProcessorParents(String [] parents) { - final Set sourceTopics = new HashSet<>(); + private Set findSourcesForProcessorParents(final String[] parents) { + final Set sourceNodes = new HashSet<>(); for (String parent : parents) { - NodeFactory nodeFactory = nodeFactories.get(parent); + final NodeFactory nodeFactory = nodeFactories.get(parent); if (nodeFactory instanceof SourceNodeFactory) { - sourceTopics.addAll(Arrays.asList(((SourceNodeFactory) nodeFactory).getTopics())); + sourceNodes.add((SourceNodeFactory) nodeFactory); } else if (nodeFactory instanceof ProcessorNodeFactory) { - sourceTopics.addAll(findSourceTopicsForProcessorParents(((ProcessorNodeFactory) nodeFactory).parents)); + sourceNodes.addAll(findSourcesForProcessorParents(((ProcessorNodeFactory) nodeFactory).parents)); } } - return sourceTopics; + return sourceNodes; } - private void connectStateStoreNameToSourceTopics(final String stateStoreName, - final ProcessorNodeFactory processorNodeFactory) { + private void connectStateStoreNameToSourceTopicsOrPattern(final String stateStoreName, + final ProcessorNodeFactory processorNodeFactory) { // we should never update the mapping from state store names to source topics if the store name already exists // in the map; this scenario is possible, for example, that a state store underlying a source KTable is // connecting to a join operator whose source topic is not the original KTable's source topic but an internal repartition topic. - if (stateStoreNameToSourceTopics.containsKey(stateStoreName)) { + + if (stateStoreNameToSourceTopics.containsKey(stateStoreName) || stateStoreNameToSourceRegex.containsKey(stateStoreName)) { return; } - final Set sourceTopics = findSourceTopicsForProcessorParents(processorNodeFactory.parents); - if (sourceTopics.isEmpty()) { - throw new TopologyBuilderException("can't find source topic for state store " + - stateStoreName); + final Set sourceTopics = new HashSet<>(); + final Set sourcePatterns = new HashSet<>(); + final Set sourceNodesForParent = findSourcesForProcessorParents(processorNodeFactory.parents); + + for (SourceNodeFactory sourceNodeFactory : sourceNodesForParent) { + if (sourceNodeFactory.pattern != null) { + sourcePatterns.add(sourceNodeFactory.pattern); + } else { + sourceTopics.addAll(sourceNodeFactory.topics); + } + } + + if (!sourceTopics.isEmpty()) { + stateStoreNameToSourceTopics.put(stateStoreName, + Collections.unmodifiableSet(sourceTopics)); + } + + if (!sourcePatterns.isEmpty()) { + stateStoreNameToSourceRegex.put(stateStoreName, + Collections.unmodifiableSet(sourcePatterns)); } - stateStoreNameToSourceTopics.put(stateStoreName, - Collections.unmodifiableSet(sourceTopics)); + } - private void maybeAddToResetList(Collection earliestResets, Collection latestResets, AutoOffsetReset offsetReset, T item) { + private void maybeAddToResetList(final Collection earliestResets, final Collection latestResets, final AutoOffsetReset offsetReset, final T item) { if (offsetReset != null) { switch (offsetReset) { case EARLIEST: @@ -932,8 +946,8 @@ public synchronized Map> nodeGroups() { } private Map> makeNodeGroups() { - HashMap> nodeGroups = new LinkedHashMap<>(); - HashMap> rootToNodeGroup = new HashMap<>(); + final HashMap> nodeGroups = new LinkedHashMap<>(); + final HashMap> rootToNodeGroup = new HashMap<>(); int nodeGroupId = 0; @@ -942,7 +956,7 @@ private Map> makeNodeGroups() { allSourceNodes.addAll(nodeToSourcePatterns.keySet()); for (String nodeName : Utils.sorted(allSourceNodes)) { - String root = nodeGrouper.root(nodeName); + final String root = nodeGrouper.root(nodeName); Set nodeGroup = rootToNodeGroup.get(root); if (nodeGroup == null) { nodeGroup = new HashSet<>(); @@ -955,7 +969,7 @@ private Map> makeNodeGroups() { // Go through non-source nodes for (String nodeName : Utils.sorted(nodeFactories.keySet())) { if (!nodeToSourceTopics.containsKey(nodeName)) { - String root = nodeGrouper.root(nodeName); + final String root = nodeGrouper.root(nodeName); Set nodeGroup = rootToNodeGroup.get(root); if (nodeGroup == null) { nodeGroup = new HashSet<>(); @@ -975,7 +989,7 @@ private Map> makeNodeGroups() { * * @see org.apache.kafka.streams.KafkaStreams#KafkaStreams(TopologyBuilder, org.apache.kafka.streams.StreamsConfig) */ - public synchronized ProcessorTopology build(Integer topicGroupId) { + public synchronized ProcessorTopology build(final Integer topicGroupId) { Set nodeGroup; if (topicGroupId != null) { nodeGroup = nodeGroups().get(topicGroupId); @@ -1013,8 +1027,8 @@ private Set globalNodeGroups() { for (String node : nodes) { final NodeFactory nodeFactory = nodeFactories.get(node); if (nodeFactory instanceof SourceNodeFactory) { - final String[] topics = ((SourceNodeFactory) nodeFactory).getTopics(); - if (topics != null && topics.length == 1 && globalTopics.contains(topics[0])) { + final List topics = ((SourceNodeFactory) nodeFactory).topics; + if (topics != null && topics.size() == 1 && globalTopics.contains(topics.get(0))) { globalGroups.addAll(nodes); } } @@ -1023,13 +1037,12 @@ private Set globalNodeGroups() { return globalGroups; } - @SuppressWarnings("unchecked") - private ProcessorTopology build(Set nodeGroup) { - List processorNodes = new ArrayList<>(nodeFactories.size()); - Map processorMap = new HashMap<>(); - Map topicSourceMap = new HashMap<>(); - Map topicSinkMap = new HashMap<>(); - Map stateStoreMap = new LinkedHashMap<>(); + private ProcessorTopology build(final Set nodeGroup) { + final List processorNodes = new ArrayList<>(nodeFactories.size()); + final Map processorMap = new HashMap<>(); + final Map topicSourceMap = new HashMap<>(); + final Map topicSinkMap = new HashMap<>(); + final Map stateStoreMap = new LinkedHashMap<>(); // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) for (NodeFactory factory : nodeFactories.values()) { @@ -1040,7 +1053,8 @@ private ProcessorTopology build(Set nodeGroup) { if (factory instanceof ProcessorNodeFactory) { for (String parent : ((ProcessorNodeFactory) factory).parents) { - processorMap.get(parent).addChild(node); + final ProcessorNode parentNode = processorMap.get(parent); + parentNode.addChild(node); } for (String stateStoreName : ((ProcessorNodeFactory) factory).stateStoreNames) { if (!stateStoreMap.containsKey(stateStoreName)) { @@ -1063,10 +1077,10 @@ private ProcessorTopology build(Set nodeGroup) { } } } else if (factory instanceof SourceNodeFactory) { - SourceNodeFactory sourceNodeFactory = (SourceNodeFactory) factory; - String[] topics = (sourceNodeFactory.pattern != null) ? + final SourceNodeFactory sourceNodeFactory = (SourceNodeFactory) factory; + final List topics = (sourceNodeFactory.pattern != null) ? sourceNodeFactory.getTopics(subscriptionUpdates.getUpdates()) : - sourceNodeFactory.getTopics(); + sourceNodeFactory.topics; for (String topic : topics) { if (internalTopicNames.contains(topic)) { @@ -1077,7 +1091,8 @@ private ProcessorTopology build(Set nodeGroup) { } } } else if (factory instanceof SinkNodeFactory) { - SinkNodeFactory sinkNodeFactory = (SinkNodeFactory) factory; + final SinkNodeFactory sinkNodeFactory = (SinkNodeFactory) factory; + for (String parent : sinkNodeFactory.parents) { processorMap.get(parent).addChild(node); if (internalTopicNames.contains(sinkNodeFactory.topic)) { @@ -1105,13 +1120,6 @@ public Map globalStateStores() { return Collections.unmodifiableMap(globalStateStores); } - private StateStore getStateStore(final String stateStoreName) { - if (stateFactories.containsKey(stateStoreName)) { - return stateFactories.get(stateStoreName).supplier.get(); - } - return globalStateStores.get(stateStoreName); - } - /** * Returns the map of topic groups keyed by the group id. * A topic group is a group of topics in the same task. @@ -1119,19 +1127,19 @@ private StateStore getStateStore(final String stateStoreName) { * @return groups of topic names */ public synchronized Map topicGroups() { - Map topicGroups = new LinkedHashMap<>(); + final Map topicGroups = new LinkedHashMap<>(); if (nodeGroups == null) nodeGroups = makeNodeGroups(); for (Map.Entry> entry : nodeGroups.entrySet()) { - Set sinkTopics = new HashSet<>(); - Set sourceTopics = new HashSet<>(); - Map internalSourceTopics = new HashMap<>(); - Map stateChangelogTopics = new HashMap<>(); + final Set sinkTopics = new HashSet<>(); + final Set sourceTopics = new HashSet<>(); + final Map internalSourceTopics = new HashMap<>(); + final Map stateChangelogTopics = new HashMap<>(); for (String node : entry.getValue()) { // if the node is a source node, add to the source topics - String[] topics = nodeToSourceTopics.get(node); + final List topics = nodeToSourceTopics.get(node); if (topics != null) { // if some of the topics are internal, add them to the internal topics for (String topic : topics) { @@ -1141,7 +1149,7 @@ public synchronized Map topicGroups() { } if (this.internalTopicNames.contains(topic)) { // prefix the internal topic name with the application id - String internalTopic = decorateTopic(topic); + final String internalTopic = decorateTopic(topic); internalSourceTopics.put(internalTopic, new InternalTopicConfig(internalTopic, Collections.singleton(InternalTopicConfig.CleanupPolicy.delete), Collections.emptyMap())); @@ -1153,7 +1161,7 @@ public synchronized Map topicGroups() { } // if the node is a sink node, add to the sink topics - String topic = nodeToSinkTopic.get(node); + final String topic = nodeToSinkTopic.get(node); if (topic != null) { if (internalTopicNames.contains(topic)) { // prefix the change log topic name with the application id @@ -1188,7 +1196,7 @@ public synchronized Map topicGroups() { private void setRegexMatchedTopicsToSourceNodes() { if (subscriptionUpdates.hasUpdates()) { for (Map.Entry stringPatternEntry : nodeToSourcePatterns.entrySet()) { - SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(stringPatternEntry.getKey()); + final SourceNodeFactory sourceNode = (SourceNodeFactory) nodeFactories.get(stringPatternEntry.getKey()); //need to update nodeToSourceTopics with topics matched from given regex nodeToSourceTopics.put(stringPatternEntry.getKey(), sourceNode.getTopics(subscriptionUpdates.getUpdates())); log.debug("nodeToSourceTopics {}", nodeToSourceTopics); @@ -1196,7 +1204,29 @@ private void setRegexMatchedTopicsToSourceNodes() { } } - private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier supplier, final String name) { + private void setRegexMatchedTopicToStateStore() { + if (subscriptionUpdates.hasUpdates()) { + for (Map.Entry> storePattern : stateStoreNameToSourceRegex.entrySet()) { + final Set updatedTopicsForStateStore = new HashSet<>(); + for (String subscriptionUpdateTopic : subscriptionUpdates.getUpdates()) { + for (Pattern pattern : storePattern.getValue()) { + if (pattern.matcher(subscriptionUpdateTopic).matches()) { + updatedTopicsForStateStore.add(subscriptionUpdateTopic); + } + } + } + if (!updatedTopicsForStateStore.isEmpty()) { + Collection storeTopics = stateStoreNameToSourceTopics.get(storePattern.getKey()); + if (storeTopics != null) { + updatedTopicsForStateStore.addAll(storeTopics); + } + stateStoreNameToSourceTopics.put(storePattern.getKey(), Collections.unmodifiableSet(updatedTopicsForStateStore)); + } + } + } + } + + private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier supplier, final String name) { if (!(supplier instanceof WindowStoreSupplier)) { return new InternalTopicConfig(name, Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), supplier.logConfig()); } @@ -1210,27 +1240,13 @@ private InternalTopicConfig createInternalTopicConfig(final StateStoreSupplier s return config; } - - /** - * Get the names of topics that are to be consumed by the source nodes created by this builder. - * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null - */ - public synchronized Set sourceTopics() { - Set topics = maybeDecorateInternalSourceTopics(sourceTopicNames); - return Collections.unmodifiableSet(topics); - } - /** * Get the Pattern to match all topics requiring to start reading from earliest available offset * @return the Pattern for matching all topics reading from earliest offset, never null */ public synchronized Pattern earliestResetTopicsPattern() { - Set topics = maybeDecorateInternalSourceTopics(earliestResetTopics); - - String[] sourceTopicNames = topics.toArray(new String[topics.size()]); - Pattern[] sourceTopicPatterns = earliestResetPatterns.toArray(new Pattern[earliestResetPatterns.size()]); - - Pattern earliestPattern = buildPatternForOffsetResetTopics(sourceTopicNames, sourceTopicPatterns); + final List topics = maybeDecorateInternalSourceTopics(earliestResetTopics); + final Pattern earliestPattern = buildPatternForOffsetResetTopics(topics, earliestResetPatterns); ensureNoRegexOverlap(earliestPattern, latestResetPatterns, latestResetTopics); @@ -1242,19 +1258,15 @@ public synchronized Pattern earliestResetTopicsPattern() { * @return the Pattern for matching all topics reading from latest offset, never null */ public synchronized Pattern latestResetTopicsPattern() { - Set topics = maybeDecorateInternalSourceTopics(latestResetTopics); - - String[] sourceTopicNames = topics.toArray(new String[topics.size()]); - Pattern[] sourceTopicPatterns = latestResetPatterns.toArray(new Pattern[latestResetPatterns.size()]); - - Pattern latestPattern = buildPatternForOffsetResetTopics(sourceTopicNames, sourceTopicPatterns); + final List topics = maybeDecorateInternalSourceTopics(latestResetTopics); + final Pattern latestPattern = buildPatternForOffsetResetTopics(topics, latestResetPatterns); ensureNoRegexOverlap(latestPattern, earliestResetPatterns, earliestResetTopics); return latestPattern; } - private void ensureNoRegexOverlap(Pattern builtPattern, Set otherPatterns, Set otherTopics) { + private void ensureNoRegexOverlap(final Pattern builtPattern, final Set otherPatterns, final Set otherTopics) { for (Pattern otherPattern : otherPatterns) { if (builtPattern.pattern().contains(otherPattern.pattern())) { @@ -1267,10 +1279,8 @@ private void ensureNoRegexOverlap(Pattern builtPattern, Set otherPatter throw new TopologyBuilderException(String.format("Found overlapping regex [%s] matching topic [%s] for a KStream with auto offset resets", builtPattern.pattern(), otherTopic)); } } - } - /** * Builds a composite pattern out of topic names and Pattern object for matching topic names. If the provided * arrays are empty a Pattern.compile("") instance is returned. @@ -1279,8 +1289,8 @@ private void ensureNoRegexOverlap(Pattern builtPattern, Set otherPatter * @param sourcePatterns Patterns for matching source topics to add to a composite pattern * @return a Pattern that is composed of the literal source topic names and any Patterns for matching source topics */ - private static synchronized Pattern buildPatternForOffsetResetTopics(String[] sourceTopics, Pattern[] sourcePatterns) { - StringBuilder builder = new StringBuilder(); + private static synchronized Pattern buildPatternForOffsetResetTopics(final Collection sourceTopics, final Collection sourcePatterns) { + final StringBuilder builder = new StringBuilder(); for (String topic : sourceTopics) { builder.append(topic).append("|"); @@ -1301,11 +1311,10 @@ private static synchronized Pattern buildPatternForOffsetResetTopics(String[] so /** * @return a mapping from state store name to a Set of source Topics. */ - public Map> stateStoreNameToSourceTopics() { - final Map> results = new HashMap<>(); + public Map> stateStoreNameToSourceTopics() { + final Map> results = new HashMap<>(); for (Map.Entry> entry : stateStoreNameToSourceTopics.entrySet()) { results.put(entry.getKey(), maybeDecorateInternalSourceTopics(entry.getValue())); - } return results; } @@ -1317,11 +1326,11 @@ public Map> stateStoreNameToSourceTopics() { * @return groups of topic names */ public synchronized Collection> copartitionGroups() { - List> list = new ArrayList<>(copartitionSourceGroups.size()); + final List> list = new ArrayList<>(copartitionSourceGroups.size()); for (Set nodeNames : copartitionSourceGroups) { Set copartitionGroup = new HashSet<>(); for (String node : nodeNames) { - String[] topics = nodeToSourceTopics.get(node); + final List topics = nodeToSourceTopics.get(node); if (topics != null) copartitionGroup.addAll(maybeDecorateInternalSourceTopics(topics)); } @@ -1330,12 +1339,8 @@ public synchronized Collection> copartitionGroups() { return Collections.unmodifiableList(list); } - private Set maybeDecorateInternalSourceTopics(final Set sourceTopics) { - return maybeDecorateInternalSourceTopics(sourceTopics.toArray(new String[sourceTopics.size()])); - } - - private Set maybeDecorateInternalSourceTopics(String ... sourceTopics) { - final Set decoratedTopics = new HashSet<>(); + private List maybeDecorateInternalSourceTopics(final Collection sourceTopics) { + final List decoratedTopics = new ArrayList<>(); for (String topic : sourceTopics) { if (internalTopicNames.contains(topic)) { decoratedTopics.add(decorateTopic(topic)); @@ -1346,7 +1351,7 @@ private Set maybeDecorateInternalSourceTopics(String ... sourceTopics) { return decoratedTopics; } - private String decorateTopic(String topic) { + private String decorateTopic(final String topic) { if (applicationId == null) { throw new TopologyBuilderException("there are internal topics and " + "applicationId hasn't been set. Call " @@ -1357,29 +1362,25 @@ private String decorateTopic(String topic) { } public synchronized Pattern sourceTopicPattern() { - if (this.topicPattern == null && !nodeToSourcePatterns.isEmpty()) { - - List allNodeToSourceTopics = new ArrayList<>(); + if (this.topicPattern == null) { + final List allSourceTopics = new ArrayList<>(); if (!nodeToSourceTopics.isEmpty()) { - for (String[] topics : nodeToSourceTopics.values()) { - allNodeToSourceTopics.addAll(Arrays.asList(topics)); - + for (List topics : nodeToSourceTopics.values()) { + allSourceTopics.addAll(maybeDecorateInternalSourceTopics(topics)); } } - int numPatterns = nodeToSourcePatterns.values().size(); - int numTopics = allNodeToSourceTopics.size(); - - Pattern[] patterns = nodeToSourcePatterns.values().toArray(new Pattern[numPatterns]); - String[] allTopics = allNodeToSourceTopics.toArray(new String[numTopics]); + Collections.sort(allSourceTopics); - this.topicPattern = buildPatternForOffsetResetTopics(allTopics, patterns); + this.topicPattern = buildPatternForOffsetResetTopics(allSourceTopics, nodeToSourcePatterns.values()); } + return this.topicPattern; } - public synchronized void updateSubscriptions(SubscriptionUpdates subscriptionUpdates, String threadId) { + public synchronized void updateSubscriptions(final SubscriptionUpdates subscriptionUpdates, final String threadId) { log.debug("stream-thread [{}] updating builder with {} topic(s) with possible matching regex subscription(s)", threadId, subscriptionUpdates); this.subscriptionUpdates = subscriptionUpdates; setRegexMatchedTopicsToSourceNodes(); + setRegexMatchedTopicToStateStore(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java index 6df9481331537..e00a102ab9222 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java @@ -29,6 +29,7 @@ * * @see FailOnInvalidTimestamp * @see LogAndSkipOnInvalidTimestamp + * @see UsePreviousTimeOnInvalidTimestamp */ public class WallclockTimestampExtractor implements TimestampExtractor { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index 0730c68ca0c9f..8de5d2304d05a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -52,14 +52,14 @@ public abstract class AbstractTask { /** * @throws ProcessorStateException if the state manager cannot be created */ - protected AbstractTask(TaskId id, - String applicationId, - Collection partitions, - ProcessorTopology topology, - Consumer consumer, - Consumer restoreConsumer, - boolean isStandby, - StateDirectory stateDirectory, + protected AbstractTask(final TaskId id, + final String applicationId, + final Collection partitions, + final ProcessorTopology topology, + final Consumer consumer, + final Consumer restoreConsumer, + final boolean isStandby, + final StateDirectory stateDirectory, final ThreadCache cache) { this.id = id; this.applicationId = applicationId; @@ -70,8 +70,7 @@ protected AbstractTask(TaskId id, // create the processor state manager try { - this.stateMgr = new ProcessorStateManager(applicationId, id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); - + stateMgr = new ProcessorStateManager(id, partitions, restoreConsumer, isStandby, stateDirectory, topology.storeToChangelogTopic()); } catch (IOException e) { throw new ProcessorStateException(String.format("task [%s] Error while creating the state manager", id), e); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java new file mode 100644 index 0000000000000..7b02d5b934365 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Checkpointable.java @@ -0,0 +1,27 @@ +/** + * 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 org.apache.kafka.common.TopicPartition; + +import java.util.Map; + +// Interface to indicate that an object has associated partition offsets that can be checkpointed +interface Checkpointable { + void checkpoint(final Map offsets); + Map checkpointed(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java index 75349931aa199..3819bb5a09d70 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImpl.java @@ -58,11 +58,11 @@ public class GlobalStateManagerImpl implements GlobalStateManager { private final File baseDir; private final OffsetCheckpoint checkpoint; private final Set globalStoreNames = new HashSet<>(); - private HashMap checkpointableOffsets; + private final Map checkpointableOffsets = new HashMap<>(); public GlobalStateManagerImpl(final ProcessorTopology topology, - final Consumer consumer, - final StateDirectory stateDirectory) { + final Consumer consumer, + final StateDirectory stateDirectory) { this.topology = topology; this.consumer = consumer; this.stateDirectory = stateDirectory; @@ -81,8 +81,7 @@ public Set initialize(final InternalProcessorContext processorContext) { } try { - this.checkpointableOffsets = new HashMap<>(checkpoint.read()); - checkpoint.delete(); + this.checkpointableOffsets.putAll(checkpoint.read()); } catch (IOException e) { try { stateDirectory.unlockGlobalState(); @@ -220,13 +219,14 @@ public void close(final Map offsets) throws IOException { if (closeFailed.length() > 0) { throw new ProcessorStateException("Exceptions caught during close of 1 or more global state stores\n" + closeFailed); } - writeCheckpoints(offsets); + checkpoint(offsets); } finally { stateDirectory.unlockGlobalState(); } } - private void writeCheckpoints(final Map offsets) { + @Override + public void checkpoint(final Map offsets) { if (!offsets.isEmpty()) { checkpointableOffsets.putAll(offsets); try { @@ -238,7 +238,7 @@ private void writeCheckpoints(final Map offsets) { } @Override - public Map checkpointedOffsets() { + public Map checkpointed() { return Collections.unmodifiableMap(checkpointableOffsets); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java index 40f2a3c826aa8..6da37e4c1c003 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStateUpdateTask.java @@ -67,7 +67,7 @@ public Map initialize() { } initTopology(); processorContext.initialized(); - return stateMgr.checkpointedOffsets(); + return stateMgr.checkpointed(); } @@ -89,6 +89,7 @@ public void update(final ConsumerRecord record) { public void flushState() { stateMgr.flush(processorContext); + stateMgr.checkpoint(offsets); } public void close() throws IOException { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 8bb5a3df65314..a0bdb9d9bdea0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.errors.StreamsException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,6 +27,7 @@ import java.util.Collection; import java.util.HashMap; import java.util.Map; +import java.util.Set; import java.util.concurrent.TimeUnit; public class InternalTopicManager { @@ -35,43 +37,70 @@ public class InternalTopicManager { public static final String RETENTION_MS = "retention.ms"; public static final Long WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); private static final int MAX_TOPIC_READY_TRY = 5; - + private final Time time; private final long windowChangeLogAdditionalRetention; private final int replicationFactor; private final StreamsKafkaClient streamsKafkaClient; - public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + public InternalTopicManager(final StreamsKafkaClient streamsKafkaClient, final int replicationFactor, + final long windowChangeLogAdditionalRetention, final Time time) { this.streamsKafkaClient = streamsKafkaClient; this.replicationFactor = replicationFactor; this.windowChangeLogAdditionalRetention = windowChangeLogAdditionalRetention; + this.time = time; } /** - * Prepares the set of given internal topics. If the topic with the correct number of partitions exists ignores it. For the ones with different number of - * partitions delete them and create new ones with correct number of partitons along with the non existing topics. - * @param topic + * Prepares a set of given internal topics. + * + * If a topic does not exist creates a new topic. + * If a topic with the correct number of partitions exists ignores it. + * If a topic exists already but has different number of partitions we fail and throw exception requesting user to reset the app before restarting again. */ - public void makeReady(final InternalTopicConfig topic, int numPartitions) { - - Map topics = new HashMap<>(); - topics.put(topic, numPartitions); + public void makeReady(final Map topics) { for (int i = 0; i < MAX_TOPIC_READY_TRY; i++) { try { - Collection topicMetadatas = streamsKafkaClient.fetchTopicMetadata(); - Map topicsToBeDeleted = getTopicsToBeDeleted(topics, topicMetadatas); - Map topicsToBeCreated = filterExistingTopics(topics, topicMetadatas); - topicsToBeCreated.putAll(topicsToBeDeleted); - streamsKafkaClient.deleteTopics(topicsToBeDeleted); - streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention); + final MetadataResponse metadata = streamsKafkaClient.fetchMetadata(); + final Map existingTopicPartitions = fetchExistingPartitionCountByTopic(metadata); + final Map topicsToBeCreated = validateTopicPartitions(topics, existingTopicPartitions); + if (metadata.brokers().size() < replicationFactor) { + throw new StreamsException("Found only " + metadata.brokers().size() + " brokers, " + + " but replication factor is " + replicationFactor + "." + + " Decrease replication factor for internal topics via StreamsConfig parameter \"replication.factor\"" + + " or add more brokers to your cluster."); + } + streamsKafkaClient.createTopics(topicsToBeCreated, replicationFactor, windowChangeLogAdditionalRetention, metadata); return; } catch (StreamsException ex) { - log.warn("Could not create internal topics: " + ex.getMessage() + ". Retry #" + i); + log.warn("Could not create internal topics: " + ex.getMessage() + " Retry #" + i); } + // backoff + time.sleep(100L); } throw new StreamsException("Could not create internal topics."); } + /** + * Get the number of partitions for the given topics + */ + public Map getNumPartitions(final Set topics) { + for (int i = 0; i < MAX_TOPIC_READY_TRY; i++) { + try { + final MetadataResponse metadata = streamsKafkaClient.fetchMetadata(); + final Map existingTopicPartitions = fetchExistingPartitionCountByTopic(metadata); + existingTopicPartitions.keySet().retainAll(topics); + + return existingTopicPartitions; + } catch (StreamsException ex) { + log.warn("Could not get number of partitions: " + ex.getMessage() + " Retry #" + i); + } + // backoff + time.sleep(100L); + } + throw new StreamsException("Could not get number of partitions."); + } + public void close() { try { streamsKafkaClient.close(); @@ -81,51 +110,35 @@ public void close() { } /** - * Return the non existing topics. - * - * @param topicsPartitionsMap - * @param topicsMetadata - * @return + * Check the existing topics to have correct number of partitions; and return the non existing topics to be created */ - private Map filterExistingTopics(final Map topicsPartitionsMap, Collection topicsMetadata) { - Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); - Map nonExistingTopics = new HashMap<>(); - // Add the topics that don't exist to the nonExistingTopics. + private Map validateTopicPartitions(final Map topicsPartitionsMap, + final Map existingTopicNamesPartitions) { + final Map topicsToBeCreated = new HashMap<>(); for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { - if (existingTopicNamesPartitions.get(topic.name()) == null) { - nonExistingTopics.put(topic, topicsPartitionsMap.get(topic)); - } - } - return nonExistingTopics; - } - - /** - * Return the topics that exist but have different partiton number to be deleted. - * @param topicsPartitionsMap - * @param topicsMetadata - * @return - */ - private Map getTopicsToBeDeleted(final Map topicsPartitionsMap, Collection topicsMetadata) { - Map existingTopicNamesPartitions = getExistingTopicNamesPartitions(topicsMetadata); - Map deleteTopics = new HashMap<>(); - // Add the topics that don't exist to the nonExistingTopics. - for (InternalTopicConfig topic: topicsPartitionsMap.keySet()) { - if (existingTopicNamesPartitions.get(topic.name()) != null) { - if (existingTopicNamesPartitions.get(topic.name()) != topicsPartitionsMap.get(topic)) { - deleteTopics.put(topic, topicsPartitionsMap.get(topic)); + if (existingTopicNamesPartitions.containsKey(topic.name())) { + if (!existingTopicNamesPartitions.get(topic.name()).equals(topicsPartitionsMap.get(topic))) { + throw new StreamsException("Existing internal topic " + topic.name() + " has invalid partitions." + + " Expected: " + topicsPartitionsMap.get(topic) + " Actual: " + existingTopicNamesPartitions.get(topic.name()) + + ". Use 'kafka.tools.StreamsResetter' tool to clean up invalid topics before processing."); } + } else { + topicsToBeCreated.put(topic, topicsPartitionsMap.get(topic)); } } - return deleteTopics; + + return topicsToBeCreated; } - private Map getExistingTopicNamesPartitions(Collection topicsMetadata) { - // The names of existing topics - Map existingTopicNamesPartitions = new HashMap<>(); + private Map fetchExistingPartitionCountByTopic(final MetadataResponse metadata) { + // The names of existing topics and corresponding partition counts + final Map existingPartitionCountByTopic = new HashMap<>(); + final Collection topicsMetadata = metadata.topicMetadata(); + for (MetadataResponse.TopicMetadata topicMetadata: topicsMetadata) { - existingTopicNamesPartitions.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); + existingPartitionCountByTopic.put(topicMetadata.topic(), topicMetadata.partitionMetadata().size()); } - return existingTopicNamesPartitions; - } + return existingPartitionCountByTopic; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java index 655b8b8551dac..bfe98abff51af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -1,10 +1,10 @@ /** * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; import java.util.LinkedList; @@ -35,7 +34,7 @@ public class MinTimestampTracker implements TimestampTracker { /** * @throws NullPointerException if the element is null */ - public void addElement(Stamped elem) { + public void addElement(final Stamped elem) { if (elem == null) throw new NullPointerException(); Stamped minElem = descendingSubsequence.peekLast(); @@ -46,12 +45,19 @@ public void addElement(Stamped elem) { descendingSubsequence.offerLast(elem); } - public void removeElement(Stamped elem) { - if (elem != null && descendingSubsequence.peekFirst() == elem) + public void removeElement(final Stamped elem) { + if (elem == null) { + return; + } + + if (descendingSubsequence.peekFirst() == elem) { descendingSubsequence.removeFirst(); + } - if (descendingSubsequence.isEmpty()) + if (descendingSubsequence.isEmpty()) { lastKnownTime = elem.timestamp; + } + } public int size() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index b66e3df7b255b..fa4cc2b96ebe5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -132,9 +132,6 @@ public void process(final K key, final V value) { this.value = value; this.nodeMetrics.metrics.measureLatencyNs(time, processDelegate, nodeMetrics.nodeProcessTimeSensor); - - // record throughput - this.nodeMetrics.metrics.recordThroughput(nodeMetrics.nodeThroughputSensor, 1); } public void punctuate(long timestamp) { @@ -174,7 +171,7 @@ protected class NodeMetrics { final Sensor nodeProcessTimeSensor; final Sensor nodePunctuateTimeSensor; - final Sensor nodeThroughputSensor; + final Sensor sourceNodeForwardSensor; final Sensor nodeCreationSensor; final Sensor nodeDestructionSensor; @@ -189,18 +186,18 @@ public NodeMetrics(StreamsMetrics metrics, String name, String sensorNamePrefix) this.metricTags.put(tagKey, tagValue); // these are all latency metrics - this.nodeProcessTimeSensor = metrics.addLatencySensor(scope, sensorNamePrefix + "." + name, "process", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); - this.nodePunctuateTimeSensor = metrics.addLatencySensor(scope, sensorNamePrefix + "." + name, "punctuate", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); - this.nodeCreationSensor = metrics.addLatencySensor(scope, sensorNamePrefix + "." + name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); - this.nodeDestructionSensor = metrics.addLatencySensor(scope, sensorNamePrefix + "." + name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); - this.nodeThroughputSensor = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "process-throughput", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); + this.nodeProcessTimeSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "process", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); + this.nodePunctuateTimeSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "punctuate", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); + this.nodeCreationSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "create", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); + this.nodeDestructionSensor = metrics.addLatencyAndThroughputSensor(scope, sensorNamePrefix + "." + name, "destroy", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); + this.sourceNodeForwardSensor = metrics.addThroughputSensor(scope, sensorNamePrefix + "." + name, "forward", Sensor.RecordingLevel.DEBUG, tagKey, tagValue); } public void removeAllSensors() { metrics.removeSensor(nodeProcessTimeSensor); metrics.removeSensor(nodePunctuateTimeSensor); - metrics.removeSensor(nodeThroughputSensor); + metrics.removeSensor(sourceNodeForwardSensor); metrics.removeSensor(nodeCreationSensor); metrics.removeSensor(nodeDestructionSensor); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index a21c3e8dc4f92..ea074da9f20d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -68,14 +68,14 @@ public class ProcessorStateManager implements StateManager { // TODO: this map does not work with customized grouper where multiple partitions // of the same topic can be assigned to the same topic. private final Map partitionForTopic; + private final OffsetCheckpoint checkpoint; /** * @throws LockException if the state directory cannot be locked because another thread holds the lock * (this might be recoverable by retrying) * @throws IOException if any severe error happens while creating or locking the state directory */ - public ProcessorStateManager(final String applicationId, - final TaskId taskId, + public ProcessorStateManager(final TaskId taskId, final Collection sources, final Consumer restoreConsumer, final boolean isStandby, @@ -83,7 +83,7 @@ public ProcessorStateManager(final String applicationId, final Map storeToChangelogTopic) throws LockException, IOException { this.taskId = taskId; this.stateDirectory = stateDirectory; - this.baseDir = stateDirectory.directoryForTask(taskId); + this.logPrefix = String.format("task [%s]", taskId); this.partitionForTopic = new HashMap<>(); for (TopicPartition source : sources) { this.partitionForTopic.put(source.topic(), source); @@ -97,18 +97,23 @@ public ProcessorStateManager(final String applicationId, this.restoreCallbacks = isStandby ? new HashMap() : null; this.storeToChangelogTopic = storeToChangelogTopic; - this.logPrefix = String.format("task [%s]", taskId); - if (!stateDirectory.lock(taskId, 5)) { - throw new LockException(String.format("%s Failed to lock the state directory: %s", logPrefix, baseDir.getCanonicalPath())); + throw new LockException(String.format("%s Failed to lock the state directory for task %s", + logPrefix, taskId)); + } + // get a handle on the parent/base directory of the task directory + // note that the parent directory could have been accidentally deleted here, + // so catch that exception if that is the case + try { + this.baseDir = stateDirectory.directoryForTask(taskId); + } catch (ProcessorStateException e) { + throw new LockException(String.format("%s Failed to get the directory for task %s. Exception %s", + logPrefix, taskId, e)); } // load the checkpoint information - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); this.checkpointedOffsets = new HashMap<>(checkpoint.read()); - - // delete the checkpoint file after finish loading its stored offsets - checkpoint.delete(); } @@ -220,6 +225,10 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest } else { restoreConsumer.seekToBeginning(singleton(storePartition)); } + log.debug("restoring partition {} from offset {} to endOffset {}", + storePartition, + restoreConsumer.position(storePartition), + endOffset); // restore its state from changelog records long limit = offsetLimit(storePartition); @@ -252,7 +261,7 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest } } - public Map checkpointedOffsets() { + public Map checkpointed() { Map partitionsAndOffsets = new HashMap<>(); for (Map.Entry entry : restoreCallbacks.entrySet()) { @@ -349,30 +358,7 @@ public void close(Map ackedOffsets) throws IOException { } if (ackedOffsets != null) { - Map checkpointOffsets = new HashMap<>(); - for (String storeName : stores.keySet()) { - // only checkpoint the offset to the offsets file if - // it is persistent AND changelog enabled - if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { - String changelogTopic = storeToChangelogTopic.get(storeName); - TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); - - Long offset = ackedOffsets.get(topicPartition); - - if (offset != null) { - // store the last offset + 1 (the log position after restoration) - checkpointOffsets.put(topicPartition, offset + 1); - } else { - // if no record was produced. we need to check the restored offset. - offset = restoredOffsets.get(topicPartition); - if (offset != null) - checkpointOffsets.put(topicPartition, offset); - } - } - } - // write the checkpoint file before closing, to indicate clean shutdown - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); - checkpoint.write(checkpointOffsets); + checkpoint(ackedOffsets); } } @@ -382,6 +368,31 @@ public void close(Map ackedOffsets) throws IOException { } } + // write the checkpoint + @Override + public void checkpoint(final Map ackedOffsets) { + for (String storeName : stores.keySet()) { + // only checkpoint the offset to the offsets file if + // it is persistent AND changelog enabled + if (stores.get(storeName).persistent() && storeToChangelogTopic.containsKey(storeName)) { + final String changelogTopic = storeToChangelogTopic.get(storeName); + final TopicPartition topicPartition = new TopicPartition(changelogTopic, getPartition(storeName)); + if (ackedOffsets.containsKey(topicPartition)) { + // store the last offset + 1 (the log position after restoration) + checkpointedOffsets.put(topicPartition, ackedOffsets.get(topicPartition) + 1); + } else if (restoredOffsets.containsKey(topicPartition)) { + checkpointedOffsets.put(topicPartition, restoredOffsets.get(topicPartition)); + } + } + } + // write the checkpoint file before closing, to indicate clean shutdown + try { + checkpoint.write(checkpointedOffsets); + } catch (IOException e) { + log.warn("Failed to write checkpoint file to {}", new File(baseDir, CHECKPOINT_FILE_NAME), e); + } + } + private int getPartition(String topic) { TopicPartition partition = partitionForTopic.get(topic); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index ec20d46b5531d..730b37dd6c42b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.StreamPartitioner; @@ -24,9 +23,21 @@ import java.util.Map; public interface RecordCollector { - void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer); - - void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, + void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer); + + void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, StreamPartitioner partitioner); void flush(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java index afb899919003b..16365788b1210 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollectorImpl.java @@ -40,7 +40,7 @@ public class RecordCollectorImpl implements RecordCollector { private static final long SEND_RETRY_BACKOFF = 100L; private static final Logger log = LoggerFactory.getLogger(RecordCollectorImpl.class); - + private final Producer producer; private final Map offsets; private final String logPrefix; @@ -54,26 +54,36 @@ public RecordCollectorImpl(Producer producer, String streamTaskI } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - send(record, keySerializer, valueSerializer, null); + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + send(topic, key, value, partition, timestamp, keySerializer, valueSerializer, null); } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, - StreamPartitioner partitioner) { + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, + StreamPartitioner partitioner) { checkForException(); - byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); - byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); - Integer partition = record.partition(); + byte[] keyBytes = keySerializer.serialize(topic, key); + byte[] valBytes = valueSerializer.serialize(topic, value); if (partition == null && partitioner != null) { - List partitions = this.producer.partitionsFor(record.topic()); + List partitions = this.producer.partitionsFor(topic); if (partitions != null && partitions.size() > 0) - partition = partitioner.partition(record.key(), record.value(), partitions.size()); + partition = partitioner.partition(key, value, partitions.size()); } ProducerRecord serializedRecord = - new ProducerRecord<>(record.topic(), partition, record.timestamp(), keyBytes, valBytes); - final String topic = serializedRecord.topic(); + new ProducerRecord<>(topic, partition, timestamp, keyBytes, valBytes); for (int attempt = 1; attempt <= MAX_SEND_ATTEMPTS; attempt++) { try { @@ -87,8 +97,10 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); offsets.put(tp, metadata.offset()); } else { - sendException = exception; - log.error("{} Error sending record to topic {}. No more offsets will be recorded for this task and the exception will eventually be thrown", logPrefix, topic, exception); + if (sendException == null) { + sendException = exception; + log.error("{} Error sending record to topic {}. No more offsets will be recorded for this task and the exception will eventually be thrown", logPrefix, topic, exception); + } } } }); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 0ebfda71c1e98..f144079416fc4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.internals.ChangedSerializer; @@ -29,11 +28,11 @@ public class SinkNode extends ProcessorNode { private final String topic; private Serializer keySerializer; private Serializer valSerializer; - private final StreamPartitioner partitioner; + private final StreamPartitioner partitioner; private ProcessorContext context; - public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { + public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner) { super(name); this.topic = topic; @@ -77,7 +76,7 @@ public void process(final K key, final V value) { } try { - collector.send(new ProducerRecord<>(topic, null, timestamp, key, value), keySerializer, valSerializer, partitioner); + collector.send(topic, key, value, null, timestamp, keySerializer, valSerializer, partitioner); } catch (ClassCastException e) { throw new StreamsException( String.format("A serializer (key: %s / value: %s) is not compatible to the actual key or value type " + diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 771f5041a2f5a..d44929c2cc7f7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -21,14 +21,17 @@ import org.apache.kafka.streams.kstream.internals.ChangedDeserializer; import org.apache.kafka.streams.processor.ProcessorContext; +import java.util.List; + public class SourceNode extends ProcessorNode { + private final List topics; + + private ProcessorContext context; private Deserializer keyDeserializer; private Deserializer valDeserializer; - private ProcessorContext context; - private String[] topics; - public SourceNode(String name, String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { + public SourceNode(String name, List topics, Deserializer keyDeserializer, Deserializer valDeserializer) { super(name); this.topics = topics; this.keyDeserializer = keyDeserializer; @@ -65,7 +68,7 @@ public void init(ProcessorContext context) { @Override public void process(final K key, final V value) { context.forward(key, value); - nodeMetrics.nodeThroughputSensor.record(); + nodeMetrics.sourceNodeForwardSensor.record(); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 954c7107815f2..82f2714be9cb6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.StreamsConfig; @@ -33,12 +32,24 @@ class StandbyContextImpl extends AbstractProcessorContext implements RecordColle private static final RecordCollector NO_OP_COLLECTOR = new RecordCollector() { @Override - public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { - + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { } @Override - public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer, final StreamPartitioner partitioner) { + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, + StreamPartitioner partitioner) { } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java index 4437a1955e700..a27098c34f22e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java @@ -51,14 +51,15 @@ public class StandbyTask extends AbstractTask { * @param metrics the {@link StreamsMetrics} created by the thread * @param stateDirectory the {@link StateDirectory} created by the thread */ - public StandbyTask(TaskId id, - String applicationId, - Collection partitions, - ProcessorTopology topology, - Consumer consumer, - Consumer restoreConsumer, - StreamsConfig config, - StreamsMetrics metrics, final StateDirectory stateDirectory) { + public StandbyTask(final TaskId id, + final String applicationId, + final Collection partitions, + final ProcessorTopology topology, + final Consumer consumer, + final Consumer restoreConsumer, + final StreamsConfig config, + final StreamsMetrics metrics, + final StateDirectory stateDirectory) { super(id, applicationId, partitions, topology, consumer, restoreConsumer, true, stateDirectory, null); // initialize the topology with its own context @@ -67,9 +68,9 @@ public StandbyTask(TaskId id, log.info("standby-task [{}] Initializing state stores", id()); initializeStateStores(); - ((StandbyContextImpl) this.processorContext).initialized(); + this.processorContext.initialized(); - this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointedOffsets()); + this.checkpointedOffsets = Collections.unmodifiableMap(stateMgr.checkpointed()); } public Map checkpointedOffsets() { @@ -92,7 +93,7 @@ public List> update(TopicPartition partition, Lis public void commit() { log.debug("standby-task [{}] Committing its state", id()); stateMgr.flush(processorContext); - + stateMgr.checkpoint(Collections.emptyMap()); // reinitialize offset limits initializeOffsetLimits(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java index d264b2606172c..b081e277d036a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateDirectory.java @@ -95,11 +95,18 @@ public File globalStateDir() { * @throws IOException */ public boolean lock(final TaskId taskId, int retry) throws IOException { + final File lockFile; // we already have the lock so bail out here if (locks.containsKey(taskId)) { return true; } - final File lockFile = new File(directoryForTask(taskId), LOCK_FILE_NAME); + try { + lockFile = new File(directoryForTask(taskId), LOCK_FILE_NAME); + } catch (ProcessorStateException e) { + // directoryForTask could be throwing an exception if another thread + // has concurrently deleted the directory + return false; + } final FileChannel channel; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java index 7343c857fa68b..3102b77163086 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StateManager.java @@ -24,7 +24,7 @@ import java.io.IOException; import java.util.Map; -interface StateManager { +interface StateManager extends Checkpointable { File baseDir(); void register(final StateStore store, final boolean loggingEnabled, final StateRestoreCallback stateRestoreCallback); @@ -36,6 +36,4 @@ interface StateManager { StateStore getGlobalStore(final String name); StateStore getStore(final String name); - - Map checkpointedOffsets(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 7b48a6fdb784c..a50a81914cf9b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -25,6 +25,7 @@ import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.config.ConfigException; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.TaskAssignmentException; @@ -58,8 +59,9 @@ public class StreamPartitionAssignor implements PartitionAssignor, Configurable { private static final Logger log = LoggerFactory.getLogger(StreamPartitionAssignor.class); + private Time time = Time.SYSTEM; - public final static int UNKNOWN = -1; + private final static int UNKNOWN = -1; public final static int NOT_AVAILABLE = -2; private static class AssignedPartition implements Comparable { @@ -124,7 +126,7 @@ public String toString() { } } - private static class InternalTopicMetadata { + static class InternalTopicMetadata { public final InternalTopicConfig config; public int numPartitions; @@ -159,7 +161,16 @@ public int compare(TopicPartition p1, TopicPartition p2) { private Map> standbyTasks; private Map> activeTasks; - InternalTopicManager internalTopicManager; + private InternalTopicManager internalTopicManager; + private CopartitionedTopicsValidator copartitionedTopicsValidator; + + /** + * Package-private method to set the time. Used for tests. + * @param time Time to be used. + */ + void time(final Time time) { + this.time = time; + } /** * We need to have the PartitionAssignor and its StreamThread to be mutually accessible @@ -210,7 +221,9 @@ public void configure(Map configs) { configs.containsKey(StreamsConfig.REPLICATION_FACTOR_CONFIG) ? (Integer) configs.get(StreamsConfig.REPLICATION_FACTOR_CONFIG) : 1, configs.containsKey(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) ? (Long) configs.get(StreamsConfig.WINDOW_STORE_CHANGE_LOG_ADDITIONAL_RETENTION_MS_CONFIG) - : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT); + : WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT, time); + + this.copartitionedTopicsValidator = new CopartitionedTopicsValidator(streamThread.getName()); } @Override @@ -263,7 +276,6 @@ public Subscription subscription(Set topics) { */ @Override public Map assign(Cluster metadata, Map subscriptions) { - // construct the client metadata from the decoded subscription info Map clientsMetadata = new HashMap<>(); @@ -369,9 +381,7 @@ public Map assign(Cluster metadata, Map topicPartitions) { + private void prepareTopic(final Map topicPartitions) { log.debug("stream-thread [{}] Starting to validate internal topics in partition assignor.", streamThread.getName()); - // if ZK is specified, prepare the internal source topic before calling partition grouper - if (internalTopicManager != null) { - for (Map.Entry entry : topicPartitions.entrySet()) { - InternalTopicConfig topic = entry.getValue().config; - Integer numPartitions = entry.getValue().numPartitions; + // first construct the topics to make ready + Map topicsToMakeReady = new HashMap<>(); + Set topicNamesToMakeReady = new HashSet<>(); - if (numPartitions == NOT_AVAILABLE) { - continue; - } - if (numPartitions < 0) { - throw new TopologyBuilderException(String.format("stream-thread [%s] Topic [%s] number of partitions not defined", streamThread.getName(), topic.name())); - } + for (InternalTopicMetadata metadata : topicPartitions.values()) { + InternalTopicConfig topic = metadata.config; + Integer numPartitions = metadata.numPartitions; - internalTopicManager.makeReady(topic, numPartitions); - - // wait until the topic metadata has been propagated to all brokers - List partitions; - do { - partitions = streamThread.restoreConsumer.partitionsFor(topic.name()); - } while (partitions == null || partitions.size() != numPartitions); + if (numPartitions == NOT_AVAILABLE) { + continue; } - } else { - List missingTopics = new ArrayList<>(); - for (String topic : topicPartitions.keySet()) { - List partitions = streamThread.restoreConsumer.partitionsFor(topic); - if (partitions == null) { - missingTopics.add(topic); - } + if (numPartitions < 0) { + throw new TopologyBuilderException(String.format("stream-thread [%s] Topic [%s] number of partitions not defined", streamThread.getName(), topic.name())); } - if (!missingTopics.isEmpty()) { - log.warn("stream-thread [{}] Topic {} do not exists but couldn't created as the config '{}' isn't supplied", - streamThread.getName(), missingTopics, StreamsConfig.ZOOKEEPER_CONNECT_CONFIG); + topicsToMakeReady.put(topic, numPartitions); + topicNamesToMakeReady.add(topic.name()); + } + + if (!topicsToMakeReady.isEmpty()) { + internalTopicManager.makeReady(topicsToMakeReady); + + // wait until each one of the topic metadata has been propagated to at least one broker + while (!allTopicsCreated(topicNamesToMakeReady, topicsToMakeReady)) { + try { + Thread.sleep(50L); + } catch (InterruptedException e) { + // ignore + } } } log.info("stream-thread [{}] Completed validating internal topics in partition assignor", streamThread.getName()); } - private void ensureCopartitioning(Collection> copartitionGroups, - Map allRepartitionTopicsNumPartitions, - Cluster metadata) { - for (Set copartitionGroup : copartitionGroups) { - ensureCopartitioning(copartitionGroup, allRepartitionTopicsNumPartitions, metadata); + private boolean allTopicsCreated(final Set topicNamesToMakeReady, final Map topicsToMakeReady) { + final Map partitions = internalTopicManager.getNumPartitions(topicNamesToMakeReady); + for (Map.Entry entry : topicsToMakeReady.entrySet()) { + final Integer numPartitions = partitions.get(entry.getKey().name()); + if (numPartitions == null || !numPartitions.equals(entry.getValue())) { + return false; + } } + return true; } - private void ensureCopartitioning(Set copartitionGroup, + private void ensureCopartitioning(Collection> copartitionGroups, Map allRepartitionTopicsNumPartitions, Cluster metadata) { - int numPartitions = UNKNOWN; - - for (String topic : copartitionGroup) { - if (!allRepartitionTopicsNumPartitions.containsKey(topic)) { - Integer partitions = metadata.partitionCountForTopic(topic); - - if (partitions == null) - throw new TopologyBuilderException(String.format("stream-thread [%s] Topic not found: %s", streamThread.getName(), topic)); - - if (numPartitions == UNKNOWN) { - numPartitions = partitions; - } else if (numPartitions != partitions) { - String[] topics = copartitionGroup.toArray(new String[copartitionGroup.size()]); - Arrays.sort(topics); - throw new TopologyBuilderException(String.format("stream-thread [%s] Topics not co-partitioned: [%s]", streamThread.getName(), Utils.mkString(Arrays.asList(topics), ","))); - } - } - } - - // if all topics for this co-partition group is repartition topics, - // then set the number of partitions to be the maximum of the number of partitions. - if (numPartitions == UNKNOWN) { - for (Map.Entry entry: allRepartitionTopicsNumPartitions.entrySet()) { - if (copartitionGroup.contains(entry.getKey())) { - int partitions = entry.getValue().numPartitions; - if (partitions > numPartitions) { - numPartitions = partitions; - } - } - } - } - // enforce co-partitioning restrictions to repartition topics by updating their number of partitions - for (Map.Entry entry : allRepartitionTopicsNumPartitions.entrySet()) { - if (copartitionGroup.contains(entry.getKey())) { - entry.getValue().numPartitions = numPartitions; - } + for (Set copartitionGroup : copartitionGroups) { + copartitionedTopicsValidator.validate(copartitionGroup, allRepartitionTopicsNumPartitions, metadata); } } @@ -748,4 +723,59 @@ public String toString() { public void close() { internalTopicManager.close(); } + + static class CopartitionedTopicsValidator { + private final String threadName; + + CopartitionedTopicsValidator(final String threadName) { + this.threadName = threadName; + } + + void validate(final Set copartitionGroup, + final Map allRepartitionTopicsNumPartitions, + final Cluster metadata) { + int numPartitions = UNKNOWN; + + for (final String topic : copartitionGroup) { + if (!allRepartitionTopicsNumPartitions.containsKey(topic)) { + final Integer partitions = metadata.partitionCountForTopic(topic); + + if (partitions == null) { + throw new TopologyBuilderException(String.format("stream-thread [%s] Topic not found: %s", threadName, topic)); + } + + if (numPartitions == UNKNOWN) { + numPartitions = partitions; + } else if (numPartitions != partitions) { + final String[] topics = copartitionGroup.toArray(new String[copartitionGroup.size()]); + Arrays.sort(topics); + throw new TopologyBuilderException(String.format("stream-thread [%s] Topics not co-partitioned: [%s]", threadName, Utils.mkString(Arrays.asList(topics), ","))); + } + } else if (allRepartitionTopicsNumPartitions.get(topic).numPartitions == NOT_AVAILABLE) { + numPartitions = NOT_AVAILABLE; + break; + } + } + + // if all topics for this co-partition group is repartition topics, + // then set the number of partitions to be the maximum of the number of partitions. + if (numPartitions == UNKNOWN) { + for (Map.Entry entry: allRepartitionTopicsNumPartitions.entrySet()) { + if (copartitionGroup.contains(entry.getKey())) { + final int partitions = entry.getValue().numPartitions; + if (partitions > numPartitions) { + numPartitions = partitions; + } + } + } + } + // enforce co-partitioning restrictions to repartition topics by updating their number of partitions + for (Map.Entry entry : allRepartitionTopicsNumPartitions.entrySet()) { + if (copartitionGroup.contains(entry.getKey())) { + entry.getValue().numPartitions = numPartitions; + } + } + + } + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index b71422148c173..3270596e2dbd3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -74,8 +74,9 @@ public void run() { log.trace("{} Start flushing its producer's sent records upon committing its state", logPrefix); // 2) flush produced records in the downstream and change logs of local states recordCollector.flush(); - - // 3) commit consumed offsets if it is dirty already + // 3) write checkpoints for any local state + stateMgr.checkpoint(recordCollectorOffsets()); + // 4) commit consumed offsets if it is dirty already commitOffsets(); } }; @@ -408,7 +409,7 @@ protected class TaskMetrics { public TaskMetrics(StreamsMetrics metrics) { String name = id.toString(); this.metrics = (StreamsMetricsImpl) metrics; - this.taskCommitTimeSensor = metrics.addLatencySensor("task", name, "commit", Sensor.RecordingLevel.DEBUG, "streams-task-id", name); + this.taskCommitTimeSensor = metrics.addLatencyAndThroughputSensor("task", name, "commit", Sensor.RecordingLevel.DEBUG, "streams-task-id", name); } public void removeAllSensors() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 38961f23f617b..a0ee9a898fc72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -23,7 +23,7 @@ import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.NoOffsetForPartitionException; +import org.apache.kafka.clients.consumer.InvalidOffsetException; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; @@ -76,33 +76,34 @@ public class StreamThread extends Thread { * A thread must only be in one state at a time * The expected state transitions with the following defined states is: * - * +-----------+ - * |Not Running|<---------------+ - * +-----+-----+ | - * | | - * v | - * +-----+-----+ | - * +-----| Running |<------------+ | - * | +-----+-----+ | | - * | | | | - * | v | | - * | +-----+------------+ | | - * <---- |Partitions | | | - * | |Revoked | | | - * | +-----+------------+ | | - * | | | | - * | v | | - * | +-----+------------+ | | - * | |Assigning | | | - * | |Partitions |------+ | - * | +-----+------------+ | - * | | | - * | | | - * | +------v---------+ | - * +--->|Pending |------------+ - * |Shutdown | - * +-----+----------+ - * + *

      +     *                +-------------+
      +     *                | Not Running | <-------+
      +     *                +-----+-------+         |
      +     *                      |                 |
      +     *                      v                 |
      +     *                +-----+-------+         |
      +     *          +<--- | Running     | <----+  |
      +     *          |     +-----+-------+      |  |
      +     *          |           |              |  |
      +     *          |           v              |  |
      +     *          |     +-----+-------+      |  |
      +     *          +<--- | Partitions  |      |  |
      +     *          |     | Revoked     |      |  |
      +     *          |     +-----+-------+      |  |
      +     *          |           |              |  |
      +     *          |           v              |  |
      +     *          |     +-----+-------+      |  |
      +     *          |     | Assigning   |      |  |
      +     *          |     | Partitions  | ---->+  |
      +     *          |     +-----+-------+         |
      +     *          |           |                 |
      +     *          |           v                 |
      +     *          |     +-----+-------+         |
      +     *          +---> | Pending     | ------->+
      +     *                | Shutdown    |
      +     *                +-------------+
      +     * 
      */ public enum State { NOT_RUNNING(1), RUNNING(1, 2, 4), PARTITIONS_REVOKED(3, 4), ASSIGNING_PARTITIONS(1, 4), PENDING_SHUTDOWN(0); @@ -157,8 +158,11 @@ public synchronized State state() { private synchronized void setState(State newState) { State oldState = state; if (!state.isValidTransition(newState)) { - log.warn("Unexpected state transition from " + state + " to " + newState); + log.warn("{} Unexpected state transition from {} to {}.", logPrefix, oldState, newState); + } else { + log.info("{} State transition from {} to {}.", logPrefix, oldState, newState); } + state = newState; if (stateListener != null) { stateListener.onChange(this, state, oldState); @@ -180,14 +184,13 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) protected final StreamsConfig config; protected final TopologyBuilder builder; - protected final Set sourceTopics; - protected final Pattern topicPattern; protected final Producer producer; protected final Consumer consumer; protected final Consumer restoreConsumer; private final String logPrefix; private final String threadClientId; + private final Pattern sourceTopicPattern; private final Map activeTasks; private final Map standbyTasks; private final Map activeTasksByPartition; @@ -200,6 +203,7 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) private final long cleanTimeMs; private final long commitTimeMs; private final StreamsMetricsThreadImpl streamsMetrics; + // TODO: this is not private only for tests, should be better refactored final StateDirectory stateDirectory; private String originalReset; private StreamPartitionAssignor partitionAssignor = null; @@ -221,13 +225,7 @@ private synchronized void setStateWhenNotInPendingShutdown(final State newState) public void onPartitionsAssigned(Collection assignment) { try { - if (state == State.PENDING_SHUTDOWN) { - log.info("stream-thread [{}] New partitions [{}] assigned while shutting down.", - StreamThread.this.getName(), assignment); - } - log.info("stream-thread [{}] New partitions [{}] assigned at the end of consumer rebalance.", - StreamThread.this.getName(), assignment); - + log.info("{} at state {}: new partitions {} assigned at the end of consumer rebalance.", logPrefix, state, assignment); setStateWhenNotInPendingShutdown(State.ASSIGNING_PARTITIONS); // do this first as we may have suspended standby tasks that // will become active or vice versa @@ -247,12 +245,7 @@ public void onPartitionsAssigned(Collection assignment) { @Override public void onPartitionsRevoked(Collection assignment) { try { - if (state == State.PENDING_SHUTDOWN) { - log.info("stream-thread [{}] New partitions [{}] revoked while shutting down.", - StreamThread.this.getName(), assignment); - } - log.info("stream-thread [{}] partitions [{}] revoked at the beginning of consumer rebalance.", - StreamThread.this.getName(), assignment); + log.info("{} at state {}: partitions {} revoked at the beginning of consumer rebalance.", logPrefix, state, assignment); setStateWhenNotInPendingShutdown(State.PARTITIONS_REVOKED); lastCleanMs = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned // suspend active tasks @@ -291,8 +284,7 @@ public StreamThread(TopologyBuilder builder, String threadName = getName(); this.config = config; this.builder = builder; - this.sourceTopics = builder.sourceTopics(); - this.topicPattern = builder.sourceTopicPattern(); + this.sourceTopicPattern = builder.sourceTopicPattern(); this.clientId = clientId; this.processId = processId; this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class); @@ -348,7 +340,7 @@ public StreamThread(TopologyBuilder builder, this.timerStartedMs = time.milliseconds(); this.lastCleanMs = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommitMs = timerStartedMs; - setState(state.RUNNING); + setState(State.RUNNING); } public void partitionAssignor(StreamPartitionAssignor partitionAssignor) { @@ -417,7 +409,7 @@ private void shutdown() { try { partitionAssignor.close(); } catch (Throwable e) { - log.error("stream-thread [{}] Failed to close KafkaStreamClient: ", this.getName(), e); + log.error("{} Failed to close KafkaStreamClient: ", logPrefix, e); } removeStreamTasks(); @@ -444,17 +436,21 @@ private RuntimeException unAssignChangeLogPartitions() { @SuppressWarnings("ThrowableNotThrown") private void shutdownTasksAndState() { - log.debug("{} shutdownTasksAndState: shutting down all active tasks [{}] and standby tasks [{}]", logPrefix, - activeTasks.keySet(), standbyTasks.keySet()); + log.debug("{} shutdownTasksAndState: shutting down" + + "active tasks {}, standby tasks {}, suspended tasks {}, and suspended standby tasks {}", + logPrefix, activeTasks.keySet(), standbyTasks.keySet(), + suspendedTasks.keySet(), suspendedStandbyTasks.keySet()); final AtomicReference firstException = new AtomicReference<>(null); // Close all processors in topology order - firstException.compareAndSet(null, closeAllTasks()); + firstException.compareAndSet(null, closeTasks(activeAndStandbytasks())); + firstException.compareAndSet(null, closeTasks(suspendedAndSuspendedStandbytasks())); // flush state firstException.compareAndSet(null, flushAllState()); // Close all task state managers. Don't need to set exception as all // state would have been flushed above - closeAllStateManagers(firstException.get() == null); + closeStateManagers(activeAndStandbytasks(), firstException.get() == null); + closeStateManagers(suspendedAndSuspendedStandbytasks(), firstException.get() == null); // only commit under clean exit if (cleanRun && firstException.get() == null) { firstException.set(commitOffsets()); @@ -469,16 +465,18 @@ private void shutdownTasksAndState() { * soon the tasks will be assigned again */ private void suspendTasksAndState() { - log.debug("{} suspendTasksAndState: suspending all active tasks [{}] and standby tasks [{}]", logPrefix, + log.debug("{} suspendTasksAndState: suspending all active tasks {} and standby tasks {}", logPrefix, activeTasks.keySet(), standbyTasks.keySet()); final AtomicReference firstException = new AtomicReference<>(null); // Close all topology nodes - firstException.compareAndSet(null, closeAllTasksTopologies()); + firstException.compareAndSet(null, closeActiveAndStandbyTasksTopologies()); // flush state firstException.compareAndSet(null, flushAllState()); // only commit after all state has been flushed and there hasn't been an exception if (firstException.get() == null) { - firstException.set(commitOffsets()); + // TODO: currently commit failures will not be thrown to users + // while suspending tasks; this need to be re-visit after KIP-98 + commitOffsets(); } // remove the changelog partitions from restore consumer firstException.compareAndSet(null, unAssignChangeLogPartitions()); @@ -494,21 +492,20 @@ interface AbstractTaskAction { void apply(final AbstractTask task); } - private RuntimeException performOnAllTasks(final AbstractTaskAction action, - final String exceptionMessage) { + private RuntimeException performOnTasks(final List tasks, + final AbstractTaskAction action, + final String exceptionMessage) { RuntimeException firstException = null; - final List allTasks = new ArrayList(activeTasks.values()); - allTasks.addAll(standbyTasks.values()); - for (final AbstractTask task : allTasks) { + for (final AbstractTask task : tasks) { try { action.apply(task); } catch (RuntimeException t) { log.error("{} Failed while executing {} {} due to {}: ", - StreamThread.this.logPrefix, - task.getClass().getSimpleName(), - task.id(), - exceptionMessage, - t); + StreamThread.this.logPrefix, + task.getClass().getSimpleName(), + task.id(), + exceptionMessage, + t); if (firstException == null) { firstException = t; } @@ -517,8 +514,20 @@ private RuntimeException performOnAllTasks(final AbstractTaskAction action, return firstException; } - private Throwable closeAllStateManagers(final boolean writeCheckpoint) { - return performOnAllTasks(new AbstractTaskAction() { + private List activeAndStandbytasks() { + final List tasks = new ArrayList(activeTasks.values()); + tasks.addAll(standbyTasks.values()); + return tasks; + } + + private List suspendedAndSuspendedStandbytasks() { + final List tasks = new ArrayList(suspendedTasks.values()); + tasks.addAll(suspendedStandbyTasks.values()); + return tasks; + } + + private Throwable closeStateManagers(final List tasks, final boolean writeCheckpoint) { + return performOnTasks(tasks, new AbstractTaskAction() { @Override public void apply(final AbstractTask task) { log.info("{} Closing the state manager of task {}", StreamThread.this.logPrefix, task.id()); @@ -529,7 +538,7 @@ public void apply(final AbstractTask task) { private RuntimeException commitOffsets() { // Exceptions should not prevent this call from going through all shutdown steps - return performOnAllTasks(new AbstractTaskAction() { + return performOnTasks(activeAndStandbytasks(), new AbstractTaskAction() { @Override public void apply(final AbstractTask task) { log.info("{} Committing consumer offsets of task {}", StreamThread.this.logPrefix, task.id()); @@ -539,7 +548,7 @@ public void apply(final AbstractTask task) { } private RuntimeException flushAllState() { - return performOnAllTasks(new AbstractTaskAction() { + return performOnTasks(activeAndStandbytasks(), new AbstractTaskAction() { @Override public void apply(final AbstractTask task) { log.info("{} Flushing state stores of task {}", StreamThread.this.logPrefix, task.id()); @@ -566,11 +575,7 @@ private void runLoop() { boolean requiresPoll = true; boolean polledRecords = false; - if (topicPattern != null) { - consumer.subscribe(topicPattern, rebalanceListener); - } else { - consumer.subscribe(new ArrayList<>(sourceTopics), rebalanceListener); - } + consumer.subscribe(sourceTopicPattern, rebalanceListener); while (stillRunning()) { this.timerStartedMs = time.milliseconds(); @@ -585,32 +590,8 @@ private void runLoop() { try { records = consumer.poll(longPoll ? this.pollTimeMs : 0); - } catch (NoOffsetForPartitionException ex) { - TopicPartition partition = ex.partition(); - if (builder.earliestResetTopicsPattern().matcher(partition.topic()).matches()) { - log.info(String.format("stream-thread [%s] setting topic to consume from earliest offset %s", this.getName(), partition.topic())); - consumer.seekToBeginning(ex.partitions()); - } else if (builder.latestResetTopicsPattern().matcher(partition.topic()).matches()) { - consumer.seekToEnd(ex.partitions()); - log.info(String.format("stream-thread [%s] setting topic to consume from latest offset %s", this.getName(), partition.topic())); - } else { - - if (originalReset == null || (!originalReset.equals("earliest") && !originalReset.equals("latest"))) { - setState(State.PENDING_SHUTDOWN); - String errorMessage = "No valid committed offset found for input topic %s (partition %s) and no valid reset policy configured." + - " You need to set configuration parameter \"auto.offset.reset\" or specify a topic specific reset " + - "policy via KStreamBuilder#stream(StreamsConfig.AutoOffsetReset offsetReset, ...) or KStreamBuilder#table(StreamsConfig.AutoOffsetReset offsetReset, ...)"; - throw new StreamsException(String.format(errorMessage, partition.topic(), partition.partition()), ex); - } - - if (originalReset.equals("earliest")) { - consumer.seekToBeginning(ex.partitions()); - } else if (originalReset.equals("latest")) { - consumer.seekToEnd(ex.partitions()); - } - log.info(String.format("stream-thread [%s] no custom setting defined for topic %s using original config %s for offset reset", this.getName(), partition.topic(), originalReset)); - } - + } catch (final InvalidOffsetException e) { + resetInvalidOffsets(e); } if (rebalanceException != null) @@ -671,6 +652,50 @@ private void runLoop() { log.info("{} Shutting down at user request", logPrefix); } + private void resetInvalidOffsets(final InvalidOffsetException e) { + final Set partitions = e.partitions(); + final Set loggedTopics = new HashSet<>(); + final Set seekToBeginning = new HashSet<>(); + final Set seekToEnd = new HashSet<>(); + + for (final TopicPartition partition : partitions) { + if (builder.earliestResetTopicsPattern().matcher(partition.topic()).matches()) { + addToResetList(partition, seekToBeginning, "stream-thread [%s] setting topic %s to consume from %s offset", "earliest", loggedTopics); + } else if (builder.latestResetTopicsPattern().matcher(partition.topic()).matches()) { + addToResetList(partition, seekToEnd, "stream-thread [%s] setting topic %s to consume from %s offset", "latest", loggedTopics); + } else { + if (originalReset == null || (!originalReset.equals("earliest") && !originalReset.equals("latest"))) { + setState(State.PENDING_SHUTDOWN); + final String errorMessage = "No valid committed offset found for input topic %s (partition %s) and no valid reset policy configured." + + " You need to set configuration parameter \"auto.offset.reset\" or specify a topic specific reset " + + "policy via KStreamBuilder#stream(StreamsConfig.AutoOffsetReset offsetReset, ...) or KStreamBuilder#table(StreamsConfig.AutoOffsetReset offsetReset, ...)"; + throw new StreamsException(String.format(errorMessage, partition.topic(), partition.partition()), e); + } + + if (originalReset.equals("earliest")) { + addToResetList(partition, seekToBeginning, "stream-thread [%s] no custom setting defined for topic %s using original config %s for offset reset", "earliest", loggedTopics); + } else if (originalReset.equals("latest")) { + addToResetList(partition, seekToEnd, "stream-thread [%s] no custom setting defined for topic %s using original config %s for offset reset", "latest", loggedTopics); + } + } + } + + if (!seekToBeginning.isEmpty()) { + consumer.seekToBeginning(seekToBeginning); + } + if (!seekToEnd.isEmpty()) { + consumer.seekToEnd(seekToEnd); + } + } + + private void addToResetList(final TopicPartition partition, final Set partitions, final String logMessage, final String resetPolicy, final Set loggedTopics) { + final String topic = partition.topic(); + if (loggedTopics.add(topic)) { + log.info(String.format(logMessage, getName(), topic, resetPolicy)); + } + partitions.add(partition); + } + private void maybeUpdateStandbyTasks() { if (!standbyTasks.isEmpty()) { if (processStandbyRecords) { @@ -764,7 +789,7 @@ protected void maybeClean() { * Commit the states of all its tasks */ private void commitAll() { - log.trace("stream-thread [{}] Committing all its owned tasks", this.getName()); + log.trace("{} Committing all its owned tasks", logPrefix); for (StreamTask task : activeTasks.values()) { commitOne(task); } @@ -830,9 +855,9 @@ public Set cachedTasks() { } protected StreamTask createStreamTask(TaskId id, Collection partitions) { - log.info("{} Creating active task {} with assigned partitions [{}]", logPrefix, id, partitions); + log.info("{} Creating active task {} with assigned partitions {}", logPrefix, id, partitions); - streamsMetrics.taskCreationSensor.record(); + streamsMetrics.taskCreatedSensor.record(); final ProcessorTopology topology = builder.build(id.topicGroupId); final RecordCollector recordCollector = new RecordCollectorImpl(producer, id.toString()); @@ -943,9 +968,9 @@ private void addStreamTasks(Collection assignment) { } StandbyTask createStandbyTask(TaskId id, Collection partitions) { - log.info("{} Creating new standby task {} with assigned partitions [{}]", logPrefix, id, partitions); + log.info("{} Creating new standby task {} with assigned partitions {}", logPrefix, id, partitions); - streamsMetrics.taskCreationSensor.record(); + streamsMetrics.taskCreatedSensor.record(); ProcessorTopology topology = builder.build(id.topicGroupId); @@ -1014,14 +1039,14 @@ private void updateStandByTaskMaps(final Map checkpointedO } private void updateSuspendedTasks() { - log.info("{} Updating suspended tasks to contain active tasks [{}]", logPrefix, activeTasks.keySet()); + log.info("{} Updating suspended tasks to contain active tasks {}", logPrefix, activeTasks.keySet()); suspendedTasks.clear(); suspendedTasks.putAll(activeTasks); suspendedStandbyTasks.putAll(standbyTasks); } private void removeStreamTasks() { - log.info("{} Removing all active tasks [{}]", logPrefix, activeTasks.keySet()); + log.info("{} Removing all active tasks {}", logPrefix, activeTasks.keySet()); try { prevTasks.clear(); @@ -1036,31 +1061,31 @@ private void removeStreamTasks() { } private void removeStandbyTasks() { - log.info("{} Removing all standby tasks [{}]", logPrefix, standbyTasks.keySet()); + log.info("{} Removing all standby tasks {}", logPrefix, standbyTasks.keySet()); standbyTasks.clear(); standbyTasksByPartition.clear(); standbyRecords.clear(); } - private RuntimeException closeAllTasks() { - return performOnAllTasks(new AbstractTaskAction() { + private RuntimeException closeTasks(final List tasks) { + return performOnTasks(tasks, new AbstractTaskAction() { @Override public void apply(final AbstractTask task) { - log.info("{} Closing a task {}", StreamThread.this.logPrefix, task.id()); + log.info("{} Closing task {}", StreamThread.this.logPrefix, task.id()); task.close(); - streamsMetrics.taskDestructionSensor.record(); + streamsMetrics.tasksClosedSensor.record(); } }, "close"); } - private RuntimeException closeAllTasksTopologies() { - return performOnAllTasks(new AbstractTaskAction() { + private RuntimeException closeActiveAndStandbyTasksTopologies() { + return performOnTasks(activeAndStandbytasks(), new AbstractTaskAction() { @Override public void apply(final AbstractTask task) { - log.info("{} Closing a task's topology {}", StreamThread.this.logPrefix, task.id()); + log.info("{} Closing task's topology {}", StreamThread.this.logPrefix, task.id()); task.closeTopology(); - streamsMetrics.taskDestructionSensor.record(); + streamsMetrics.tasksClosedSensor.record(); } }, "close"); } @@ -1116,40 +1141,40 @@ private class StreamsMetricsThreadImpl extends StreamsMetricsImpl { final Sensor pollTimeSensor; final Sensor processTimeSensor; final Sensor punctuateTimeSensor; - final Sensor taskCreationSensor; - final Sensor taskDestructionSensor; + final Sensor taskCreatedSensor; + final Sensor tasksClosedSensor; final Sensor skippedRecordsSensor; public StreamsMetricsThreadImpl(Metrics metrics, String groupName, String prefix, Map tags) { super(metrics, groupName, tags); - this.commitTimeSensor = metrics.sensor(prefix + ".commit-time", Sensor.RecordingLevel.INFO); - this.commitTimeSensor.add(metrics.metricName("commit-time-avg", this.groupName, "The average commit time in ms", this.tags), new Avg()); - this.commitTimeSensor.add(metrics.metricName("commit-time-max", this.groupName, "The maximum commit time in ms", this.tags), new Max()); - this.commitTimeSensor.add(metrics.metricName("commit-calls-rate", this.groupName, "The average per-second number of commit calls", this.tags), new Rate(new Count())); + this.commitTimeSensor = metrics.sensor(prefix + ".commit-latency", Sensor.RecordingLevel.INFO); + this.commitTimeSensor.add(metrics.metricName("commit-latency-avg", this.groupName, "The average commit time in ms", this.tags), new Avg()); + this.commitTimeSensor.add(metrics.metricName("commit-latency-max", this.groupName, "The maximum commit time in ms", this.tags), new Max()); + this.commitTimeSensor.add(metrics.metricName("commit-rate", this.groupName, "The average per-second number of commit calls", this.tags), new Rate(new Count())); - this.pollTimeSensor = metrics.sensor(prefix + ".poll-time", Sensor.RecordingLevel.INFO); - this.pollTimeSensor.add(metrics.metricName("poll-time-avg", this.groupName, "The average poll time in ms", this.tags), new Avg()); - this.pollTimeSensor.add(metrics.metricName("poll-time-max", this.groupName, "The maximum poll time in ms", this.tags), new Max()); - this.pollTimeSensor.add(metrics.metricName("poll-calls-rate", this.groupName, "The average per-second number of record-poll calls", this.tags), new Rate(new Count())); + this.pollTimeSensor = metrics.sensor(prefix + ".poll-latency", Sensor.RecordingLevel.INFO); + this.pollTimeSensor.add(metrics.metricName("poll-latency-avg", this.groupName, "The average poll time in ms", this.tags), new Avg()); + this.pollTimeSensor.add(metrics.metricName("poll-latency-max", this.groupName, "The maximum poll time in ms", this.tags), new Max()); + this.pollTimeSensor.add(metrics.metricName("poll-rate", this.groupName, "The average per-second number of record-poll calls", this.tags), new Rate(new Count())); - this.processTimeSensor = metrics.sensor(prefix + ".process-time", Sensor.RecordingLevel.INFO); - this.processTimeSensor.add(metrics.metricName("process-time-avg", this.groupName, "The average process time in ms", this.tags), new Avg()); - this.processTimeSensor.add(metrics.metricName("process-time-max", this.groupName, "The maximum process time in ms", this.tags), new Max()); - this.processTimeSensor.add(metrics.metricName("process-calls-rate", this.groupName, "The average per-second number of process calls", this.tags), new Rate(new Count())); + this.processTimeSensor = metrics.sensor(prefix + ".process-latency", Sensor.RecordingLevel.INFO); + this.processTimeSensor.add(metrics.metricName("process-latency-avg", this.groupName, "The average process time in ms", this.tags), new Avg()); + this.processTimeSensor.add(metrics.metricName("process-latency-max", this.groupName, "The maximum process time in ms", this.tags), new Max()); + this.processTimeSensor.add(metrics.metricName("process-rate", this.groupName, "The average per-second number of process calls", this.tags), new Rate(new Count())); - this.punctuateTimeSensor = metrics.sensor(prefix + ".punctuate-time", Sensor.RecordingLevel.INFO); - this.punctuateTimeSensor.add(metrics.metricName("punctuate-time-avg", this.groupName, "The average punctuate time in ms", this.tags), new Avg()); - this.punctuateTimeSensor.add(metrics.metricName("punctuate-time-max", this.groupName, "The maximum punctuate time in ms", this.tags), new Max()); - this.punctuateTimeSensor.add(metrics.metricName("punctuate-calls-rate", this.groupName, "The average per-second number of punctuate calls", this.tags), new Rate(new Count())); + this.punctuateTimeSensor = metrics.sensor(prefix + ".punctuate-latency", Sensor.RecordingLevel.INFO); + this.punctuateTimeSensor.add(metrics.metricName("punctuate-latency-avg", this.groupName, "The average punctuate time in ms", this.tags), new Avg()); + this.punctuateTimeSensor.add(metrics.metricName("punctuate-latency-max", this.groupName, "The maximum punctuate time in ms", this.tags), new Max()); + this.punctuateTimeSensor.add(metrics.metricName("punctuate-rate", this.groupName, "The average per-second number of punctuate calls", this.tags), new Rate(new Count())); - this.taskCreationSensor = metrics.sensor(prefix + ".task-creation", Sensor.RecordingLevel.INFO); - this.taskCreationSensor.add(metrics.metricName("task-creation-rate", this.groupName, "The average per-second number of newly created tasks", this.tags), new Rate(new Count())); + this.taskCreatedSensor = metrics.sensor(prefix + ".task-created", Sensor.RecordingLevel.INFO); + this.taskCreatedSensor.add(metrics.metricName("task-created-rate", this.groupName, "The average per-second number of newly created tasks", this.tags), new Rate(new Count())); - this.taskDestructionSensor = metrics.sensor(prefix + ".task-destruction", Sensor.RecordingLevel.INFO); - this.taskDestructionSensor.add(metrics.metricName("task-destruction-rate", this.groupName, "The average per-second number of destructed tasks", this.tags), new Rate(new Count())); + this.tasksClosedSensor = metrics.sensor(prefix + ".task-closed", Sensor.RecordingLevel.INFO); + this.tasksClosedSensor.add(metrics.metricName("task-closed-rate", this.groupName, "The average per-second number of closed tasks", this.tags), new Rate(new Count())); this.skippedRecordsSensor = metrics.sensor(prefix + ".skipped-records"); - this.skippedRecordsSensor.add(metrics.metricName("skipped-records-count", this.groupName, "The average per-second number of skipped records.", this.tags), new Rate(new Count())); + this.skippedRecordsSensor.add(metrics.metricName("skipped-records-rate", this.groupName, "The average per-second number of skipped records.", this.tags), new Rate(new Count())); } @@ -1164,8 +1189,8 @@ public void removeAllSensors() { removeSensor(pollTimeSensor); removeSensor(processTimeSensor); removeSensor(punctuateTimeSensor); - removeSensor(taskCreationSensor); - removeSensor(taskDestructionSensor); + removeSensor(taskCreatedSensor); + removeSensor(tasksClosedSensor); removeSensor(skippedRecordsSensor); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java index f33a4e40f6430..bb75759d9d1cb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClient.java @@ -13,31 +13,33 @@ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.ClientResponse; import org.apache.kafka.clients.ClientUtils; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.KafkaClient; -import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.Metadata; -import org.apache.kafka.clients.ClientResponse; -import org.apache.kafka.clients.ClientRequest; +import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.metrics.JmxReporter; 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.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.ApiVersionsRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.CreateTopicsRequest; import org.apache.kafka.common.requests.CreateTopicsResponse; -import org.apache.kafka.common.requests.DeleteTopicsRequest; -import org.apache.kafka.common.requests.DeleteTopicsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; -import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.StreamsConfig; @@ -45,35 +47,55 @@ import java.io.IOException; import java.net.InetSocketAddress; +import java.util.Collections; import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.HashSet; -import java.util.List; import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; import java.util.Properties; -import java.util.Collection; import java.util.concurrent.TimeUnit; public class StreamsKafkaClient { + private static final ConfigDef CONFIG = StreamsConfig.configDef() + .withClientSslSupport() + .withClientSaslSupport(); + + public static class Config extends AbstractConfig { + + public static Config fromStreamsConfig(StreamsConfig streamsConfig) { + return new Config(streamsConfig.originals()); + } + + public Config(Map originals) { + super(CONFIG, originals, false); + } + } + private final KafkaClient kafkaClient; private final List reporters; - private final StreamsConfig streamsConfig; + private final Config streamsConfig; private static final int MAX_INFLIGHT_REQUESTS = 100; public StreamsKafkaClient(final StreamsConfig streamsConfig) { + this(Config.fromStreamsConfig(streamsConfig)); + } + public StreamsKafkaClient(final Config streamsConfig) { this.streamsConfig = streamsConfig; + final Time time = new SystemTime(); final Map metricTags = new LinkedHashMap<>(); metricTags.put("client-id", StreamsConfig.CLIENT_ID_CONFIG); - final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final Metadata metadata = new Metadata(streamsConfig.getLong( + StreamsConfig.RETRY_BACKOFF_MS_CONFIG), + streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG) + ); final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), time.milliseconds()); + metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), time.milliseconds()); final MetricConfig metricConfig = new MetricConfig().samples(streamsConfig.getInt(CommonClientConfigs.METRICS_NUM_SAMPLES_CONFIG)) .timeWindow(streamsConfig.getLong(CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG), TimeUnit.MILLISECONDS) @@ -86,17 +108,24 @@ public StreamsKafkaClient(final StreamsConfig streamsConfig) { final ChannelBuilder channelBuilder = ClientUtils.createChannelBuilder(streamsConfig.values()); - final Selector selector = new Selector(streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), metrics, time, "kafka-client", channelBuilder); + final Selector selector = new Selector( + streamsConfig.getLong(StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG), + metrics, + time, + "kafka-client", + channelBuilder); kafkaClient = new NetworkClient( - selector, - metadata, - streamsConfig.getString(StreamsConfig.CLIENT_ID_CONFIG), - MAX_INFLIGHT_REQUESTS, // a fixed large enough value will suffice - streamsConfig.getLong(StreamsConfig.RECONNECT_BACKOFF_MS_CONFIG), - streamsConfig.getInt(StreamsConfig.SEND_BUFFER_CONFIG), - streamsConfig.getInt(StreamsConfig.RECEIVE_BUFFER_CONFIG), - streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), time, true); + selector, + metadata, + streamsConfig.getString(StreamsConfig.CLIENT_ID_CONFIG), + MAX_INFLIGHT_REQUESTS, // a fixed large enough value will suffice + streamsConfig.getLong(StreamsConfig.RECONNECT_BACKOFF_MS_CONFIG), + streamsConfig.getInt(StreamsConfig.SEND_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.RECEIVE_BUFFER_CONFIG), + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG), + time, + true); } public void close() throws IOException { @@ -106,13 +135,10 @@ public void close() throws IOException { } /** - * Creates a set of new topics using batch request. - * - * @param topicsMap - * @param replicationFactor - * @param windowChangeLogAdditionalRetention + * Create a set of new topics using batch request. */ - public void createTopics(final Map topicsMap, final int replicationFactor, final long windowChangeLogAdditionalRetention) { + public void createTopics(final Map topicsMap, final int replicationFactor, + final long windowChangeLogAdditionalRetention, final MetadataResponse metadata) { final Map topicRequestDetails = new HashMap<>(); for (InternalTopicConfig internalTopicConfig:topicsMap.keySet()) { @@ -121,81 +147,47 @@ public void createTopics(final Map topicsMap, fina for (String key : topicProperties.stringPropertyNames()) { topicConfig.put(key, topicProperties.getProperty(key)); } - final CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails(topicsMap.get(internalTopicConfig), (short) replicationFactor, topicConfig); + final CreateTopicsRequest.TopicDetails topicDetails = new CreateTopicsRequest.TopicDetails( + topicsMap.get(internalTopicConfig), + (short) replicationFactor, + topicConfig); topicRequestDetails.put(internalTopicConfig.name(), topicDetails); } - final CreateTopicsRequest.Builder createTopicsRequest = - new CreateTopicsRequest.Builder(topicRequestDetails, - streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(createTopicsRequest); + + final ClientRequest clientRequest = kafkaClient.newClientRequest( + getControllerReadyBrokerId(metadata), + new CreateTopicsRequest.Builder( + topicRequestDetails, + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)), + Time.SYSTEM.milliseconds(), + true); + final ClientResponse clientResponse = sendRequest(clientRequest); + + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } if (!(clientResponse.responseBody() instanceof CreateTopicsResponse)) { - throw new StreamsException("Inconsistent response type for internal topic creation request. Expected CreateTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); + throw new StreamsException("Inconsistent response type for internal topic creation request. " + + "Expected CreateTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); } final CreateTopicsResponse createTopicsResponse = (CreateTopicsResponse) clientResponse.responseBody(); for (InternalTopicConfig internalTopicConfig : topicsMap.keySet()) { CreateTopicsResponse.Error error = createTopicsResponse.errors().get(internalTopicConfig.name()); - if (!error.is(Errors.NONE)) { - if (error.is(Errors.TOPIC_ALREADY_EXISTS)) { - continue; - } else { - throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + " due to " + error.messageWithFallback()); - } - } - } - } - - /** - * Delets a set of topics. - * - * @param topics - */ - public void deleteTopics(final Map topics) { - - final Set topicNames = new HashSet<>(); - for (InternalTopicConfig internalTopicConfig: topics.keySet()) { - topicNames.add(internalTopicConfig.name()); - } - deleteTopics(topicNames); - } - - /** - * Delete a set of topics in one request. - * - * @param topics - */ - private void deleteTopics(final Set topics) { - - final DeleteTopicsRequest.Builder deleteTopicsRequest = - new DeleteTopicsRequest.Builder(topics, streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG)); - final ClientResponse clientResponse = sendRequest(deleteTopicsRequest); - if (!(clientResponse.responseBody() instanceof DeleteTopicsResponse)) { - throw new StreamsException("Inconsistent response type for internal topic deletion request. Expected DeleteTopicsResponse but received " + clientResponse.responseBody().getClass().getName()); - } - final DeleteTopicsResponse deleteTopicsResponse = (DeleteTopicsResponse) clientResponse.responseBody(); - for (Map.Entry entry : deleteTopicsResponse.errors().entrySet()) { - if (entry.getValue() != Errors.NONE) { - throw new StreamsException("Could not delete topic: " + entry.getKey() + " due to " + entry.getValue().message()); + if (!error.is(Errors.NONE) && !error.is(Errors.TOPIC_ALREADY_EXISTS)) { + throw new StreamsException("Could not create topic: " + internalTopicConfig.name() + " due to " + error.messageWithFallback()); } } - } /** - * Send a request to kafka broker of this client. Keep polling until the corresponding response is received. * - * @param request + * @param nodes List of nodes to pick from. + * @return The first node that is ready to accept requests. */ - private ClientResponse sendRequest(final AbstractRequest.Builder request) { - + private String ensureOneNodeIsReady(final List nodes) { String brokerId = null; - - final Metadata metadata = new Metadata(streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); - final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); - metadata.update(Cluster.bootstrap(addresses), Time.SYSTEM.milliseconds()); - - final List nodes = metadata.fetch().nodes(); final long readyTimeout = Time.SYSTEM.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); boolean foundNode = false; while (!foundNode && (Time.SYSTEM.milliseconds() < readyTimeout)) { @@ -206,21 +198,56 @@ private ClientResponse sendRequest(final AbstractRequest.Builder request) { break; } } - kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); + try { + kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); + } catch (final Exception e) { + throw new StreamsException("Could not poll.", e); + } } if (brokerId == null) { throw new StreamsException("Could not find any available broker."); } + return brokerId; + } - final ClientRequest clientRequest = kafkaClient.newClientRequest( - brokerId, request, Time.SYSTEM.milliseconds(), true, null); + /** + * + * @return if Id of the controller node, or an exception if no controller is found or + * controller is not ready + */ + private String getControllerReadyBrokerId(final MetadataResponse metadata) { + return ensureOneNodeIsReady(Collections.singletonList(metadata.controller())); + } - kafkaClient.send(clientRequest, Time.SYSTEM.milliseconds()); + /** + * @return the Id of any broker that is ready, or an exception if no broker is ready. + */ + private String getAnyReadyBrokerId() { + final Metadata metadata = new Metadata( + streamsConfig.getLong(StreamsConfig.RETRY_BACKOFF_MS_CONFIG), + streamsConfig.getLong(StreamsConfig.METADATA_MAX_AGE_CONFIG)); + final List addresses = ClientUtils.parseAndValidateAddresses(streamsConfig.getList(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG)); + metadata.update(Cluster.bootstrap(addresses), Collections.emptySet(), Time.SYSTEM.milliseconds()); + final List nodes = metadata.fetch().nodes(); + return ensureOneNodeIsReady(nodes); + } + + private ClientResponse sendRequest(final ClientRequest clientRequest) { + try { + kafkaClient.send(clientRequest, Time.SYSTEM.milliseconds()); + } catch (final Exception e) { + throw new StreamsException("Could not send request.", e); + } final long responseTimeout = Time.SYSTEM.milliseconds() + streamsConfig.getInt(StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG); // Poll for the response. while (Time.SYSTEM.milliseconds() < responseTimeout) { - List responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); + final List responseList; + try { + responseList = kafkaClient.poll(streamsConfig.getLong(StreamsConfig.POLL_MS_CONFIG), Time.SYSTEM.milliseconds()); + } catch (final IllegalStateException e) { + throw new StreamsException("Could not poll.", e); + } if (!responseList.isEmpty()) { if (responseList.size() > 1) { throw new StreamsException("Sent one request but received multiple or no responses."); @@ -229,44 +256,66 @@ private ClientResponse sendRequest(final AbstractRequest.Builder request) { if (response.requestHeader().correlationId() == clientRequest.correlationId()) { return response; } else { - throw new StreamsException("Inconsistent response received from broker " + brokerId + - ", expected correlation id " + clientRequest.correlationId() + ", but received " + - response.requestHeader().correlationId()); + throw new StreamsException("Inconsistent response received from the broker " + + clientRequest.destination() + ", expected correlation id " + clientRequest.correlationId() + + ", but received " + response.requestHeader().correlationId()); } } } throw new StreamsException("Failed to get response from broker within timeout"); - } + } /** - * Get the metadata for a topic. - * @param topic - * @return + * Fetch the metadata for all topics */ - public MetadataResponse.TopicMetadata getTopicMetadata(final String topic) { + public MetadataResponse fetchMetadata() { - final ClientResponse clientResponse = sendRequest(MetadataRequest.Builder.allTopics()); + final ClientRequest clientRequest = kafkaClient.newClientRequest( + getAnyReadyBrokerId(), + new MetadataRequest.Builder(null), + Time.SYSTEM.milliseconds(), + true); + final ClientResponse clientResponse = sendRequest(clientRequest); + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } if (!(clientResponse.responseBody() instanceof MetadataResponse)) { - throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); + throw new StreamsException("Inconsistent response type for internal topic metadata request. " + + "Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); } final MetadataResponse metadataResponse = (MetadataResponse) clientResponse.responseBody(); - for (MetadataResponse.TopicMetadata topicMetadata: metadataResponse.topicMetadata()) { - if (topicMetadata.topic().equalsIgnoreCase(topic)) { - return topicMetadata; - } - } - return null; + return metadataResponse; } + /** + * Check if the used brokers have version 0.10.1.x or higher. + *

      + * Note, for pre 0.10.x brokers the broker version cannot be checked and the client will hang and retry + * until it {@link StreamsConfig#REQUEST_TIMEOUT_MS_CONFIG times out}. + * + * @throws StreamsException if brokers have version 0.10.0.x + */ + public void checkBrokerCompatibility() throws StreamsException { + final ClientRequest clientRequest = kafkaClient.newClientRequest( + getAnyReadyBrokerId(), + new ApiVersionsRequest.Builder(), + Time.SYSTEM.milliseconds(), + true); + + final ClientResponse clientResponse = sendRequest(clientRequest); + if (!clientResponse.hasResponse()) { + throw new StreamsException("Empty response for client request."); + } + if (!(clientResponse.responseBody() instanceof ApiVersionsResponse)) { + throw new StreamsException("Inconsistent response type for API versions request. " + + "Expected ApiVersionsResponse but received " + clientResponse.responseBody().getClass().getName()); + } - public Collection fetchTopicMetadata() { - final ClientResponse clientResponse = sendRequest(MetadataRequest.Builder.allTopics()); - if (!(clientResponse.responseBody() instanceof MetadataResponse)) { - throw new StreamsException("Inconsistent response type for internal topic metadata request. Expected MetadataResponse but received " + clientResponse.responseBody().getClass().getName()); + final ApiVersionsResponse apiVersionsResponse = (ApiVersionsResponse) clientResponse.responseBody(); + + if (apiVersionsResponse.apiVersion(ApiKeys.CREATE_TOPICS.id) == null) { + throw new StreamsException("Kafka Streams requires broker version 0.10.1.x or higher."); } - final MetadataResponse metadataResponse = (MetadataResponse) clientResponse.responseBody(); - return metadataResponse.topicMetadata(); } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java index a59eb5f914f00..6fb6e062a4c84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetadataState.java @@ -84,7 +84,7 @@ public synchronized Collection getAllMetadataForStore(final Str return allMetadata; } - final Set sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); + final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); if (sourceTopics == null) { return Collections.emptyList(); } @@ -201,7 +201,7 @@ public synchronized void onChange(final Map> curre rebuildMetadata(currentState); } - private boolean hasPartitionsForAnyTopics(final Set topicNames, final Set partitionForHost) { + private boolean hasPartitionsForAnyTopics(final List topicNames, final Set partitionForHost) { for (TopicPartition topicPartition : partitionForHost) { if (topicNames.contains(topicPartition.topic())) { return true; @@ -215,13 +215,13 @@ private void rebuildMetadata(final Map> currentSta if (currentState.isEmpty()) { return; } - final Map> stores = builder.stateStoreNameToSourceTopics(); + final Map> stores = builder.stateStoreNameToSourceTopics(); for (Map.Entry> entry : currentState.entrySet()) { final HostInfo key = entry.getKey(); final Set partitionsForHost = new HashSet<>(entry.getValue()); final Set storesOnHost = new HashSet<>(); - for (Map.Entry> storeTopicEntry : stores.entrySet()) { - final Set topicsForStore = storeTopicEntry.getValue(); + for (Map.Entry> storeTopicEntry : stores.entrySet()) { + final List topicsForStore = storeTopicEntry.getValue(); if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) { storesOnHost.add(storeTopicEntry.getKey()); } @@ -259,7 +259,7 @@ private StreamsMetadata getStreamsMetadataForKey(final String storeName, } private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { - final Set sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); + final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); if (sourceTopics == null || sourceTopics.isEmpty()) { return null; } @@ -271,11 +271,11 @@ private boolean isInitialized() { } private class SourceTopicsInfo { - private final Set sourceTopics; + private final List sourceTopics; private int maxPartitions; private String topicWithMostPartitions; - private SourceTopicsInfo(final Set sourceTopics) { + private SourceTopicsInfo(final List sourceTopics) { this.sourceTopics = sourceTopics; for (String topic : sourceTopics) { final List partitions = clusterMetadata.partitionsForTopic(topic); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java index e29476bd5cd87..bccf736d0a8ad 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImpl.java @@ -113,7 +113,7 @@ private Map tagMap(String... tags) { * @throws IllegalArgumentException if tags is not constructed in key-value pairs */ @Override - public Sensor addLatencySensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordingLevel, String... tags) { + public Sensor addLatencyAndThroughputSensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordingLevel, String... tags) { Map tagMap = tagMap(tags); // first add the global operation metrics if not yet, with the global tags only @@ -150,15 +150,15 @@ public Sensor addThroughputSensor(String scopeName, String entityName, String op } private void addLatencyMetrics(String scopeName, Sensor sensor, String entityName, String opName, Map tags) { - maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-avg-latency", groupNameFromScope(scopeName), + maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-latency-avg", groupNameFromScope(scopeName), "The average latency of " + entityName + " " + opName + " operation.", tags), new Avg()); - maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-max-latency", groupNameFromScope(scopeName), + maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-latency-max", groupNameFromScope(scopeName), "The max latency of " + entityName + " " + opName + " operation.", tags), new Max()); addThroughputMetrics(scopeName, sensor, entityName, opName, tags); } private void addThroughputMetrics(String scopeName, Sensor sensor, String entityName, String opName, Map tags) { - maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-qps", groupNameFromScope(scopeName), + maybeAddMetric(sensor, metrics.metricName(entityName + "-" + opName + "-rate", groupNameFromScope(scopeName), "The average number of occurrence of " + entityName + " " + opName + " operation per second.", tags), new Rate(new Count())); } @@ -166,7 +166,7 @@ private void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) if (!metrics.metrics().containsKey(name)) { sensor.add(name, stat); } else { - log.debug("Trying to add metric twice " + name); + log.trace("Trying to add metric twice: {}", name); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java index f09d357c59300..0f22b3470ecdb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlySessionStore.java @@ -33,6 +33,10 @@ public interface ReadOnlySessionStore { /** * Retrieve all aggregated sessions for the provided key + * + * For each key, the iterator guarantees ordering of sessions, starting from the oldest/earliest + * available session to the newest/latest session. + * * @param key record key to find aggregated session values for * @return KeyValueIterator containing all sessions for the provided key. */ diff --git a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java index 80da9e8f9cfa1..ff65f164069e0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/ReadOnlyWindowStore.java @@ -30,6 +30,28 @@ public interface ReadOnlyWindowStore { /** * Get all the key-value pairs with the given key and the time range from all * the existing windows. + *

      + * The time range is inclusive and applies to the starting timestamp of the window. + * For example, if we have the following windows: + *

      + *

      +     * +-------------------------------+
      +     * |  key  | start time | end time |
      +     * +-------+------------+----------+
      +     * |   A   |     10     |    20    |
      +     * +-------+------------+----------+
      +     * |   A   |     15     |    25    |
      +     * +-------+------------+----------+
      +     * |   A   |     20     |    30    |
      +     * +-------+------------+----------+
      +     * |   A   |     25     |    35    |
      +     * +--------------------------------
      +     * 
      + * And we call {@code store.fetch("A", 10, 20)} then the results will contain the first + * three windows from the table above, i.e., all those where 10 <= start time <= 20. + *

      + * For each key, the iterator guarantees ordering of windows, starting from the oldest/earliest + * available window to the newest/latest window. * * @return an iterator over key-value pairs {@code } * @throws InvalidStateStoreException if the store is not initialized diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java index b19510c0b0276..059be375917bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -32,18 +32,21 @@ public final class StateSerdes { /** * Create a new instance of {@link StateSerdes} for the given state name and key-/value-type classes. * - * @param stateName the name of the state - * @param keyClass the class of the key type - * @param valueClass the class of the value type - * @param the key type - * @param the value type - * @return a new instance of {@link StateSerdes} + * @param topic the topic name + * @param keyClass the class of the key type + * @param valueClass the class of the value type + * @param the key type + * @param the value type + * @return a new instance of {@link StateSerdes} */ - public static StateSerdes withBuiltinTypes(String stateName, Class keyClass, Class valueClass) { - return new StateSerdes<>(stateName, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); + public static StateSerdes withBuiltinTypes( + final String topic, + final Class keyClass, + final Class valueClass) { + return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); } - private final String stateName; + private final String topic; private final Serde keySerde; private final Serde valueSerde; @@ -53,22 +56,26 @@ public static StateSerdes withBuiltinTypes(String stateName, Class< * is provided to bind this serde factory to, so that future calls for serialize / deserialize do not * need to provide the topic name any more. * - * @param stateName the name of the state - * @param keySerde the serde for keys; cannot be null - * @param valueSerde the serde for values; cannot be null + * @param topic the topic name + * @param keySerde the serde for keys; cannot be null + * @param valueSerde the serde for values; cannot be null * @throws IllegalArgumentException if key or value serde is null */ @SuppressWarnings("unchecked") - public StateSerdes(String stateName, - Serde keySerde, - Serde valueSerde) { - this.stateName = stateName; - - if (keySerde == null) + public StateSerdes(final String topic, + final Serde keySerde, + final Serde valueSerde) { + if (topic == null) { + throw new IllegalArgumentException("topic cannot be null"); + } + if (keySerde == null) { throw new IllegalArgumentException("key serde cannot be null"); - if (valueSerde == null) + } + if (valueSerde == null) { throw new IllegalArgumentException("value serde cannot be null"); + } + this.topic = topic; this.keySerde = keySerde; this.valueSerde = valueSerde; } @@ -128,12 +135,12 @@ public Serializer valueSerializer() { } /** - * Return the name of the state. + * Return the topic. * - * @return the name of the state + * @return the topic */ - public String stateName() { - return stateName; + public String topic() { + return topic; } /** @@ -143,7 +150,7 @@ public String stateName() { * @return the key as typed object */ public K keyFrom(byte[] rawKey) { - return keySerde.deserializer().deserialize(stateName, rawKey); + return keySerde.deserializer().deserialize(topic, rawKey); } /** @@ -153,7 +160,7 @@ public K keyFrom(byte[] rawKey) { * @return the value as typed object */ public V valueFrom(byte[] rawValue) { - return valueSerde.deserializer().deserialize(stateName, rawValue); + return valueSerde.deserializer().deserialize(topic, rawValue); } /** @@ -163,7 +170,7 @@ public V valueFrom(byte[] rawValue) { * @return the serialized key */ public byte[] rawKey(K key) { - return keySerde.serializer().serialize(stateName, key); + return keySerde.serializer().serialize(topic, key); } /** @@ -173,6 +180,6 @@ public byte[] rawKey(K key) { * @return the serialized value */ public byte[] rawValue(V value) { - return valueSerde.serializer().serialize(stateName, value); + return valueSerde.serializer().serialize(topic, value); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java index b6e6d0c2df381..958b7788411fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreIterator.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.KeyValue; import java.io.Closeable; -import java.util.Iterator; /** * Iterator interface of {@link KeyValue} with key typed {@link Long} used for {@link WindowStore#fetch(Object, long, long)}. @@ -32,7 +31,7 @@ * * @param Type of values */ -public interface WindowStoreIterator extends Iterator>, Closeable { +public interface WindowStoreIterator extends KeyValueIterator, Closeable { @Override void close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java new file mode 100644 index 0000000000000..009dad05937c4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/AbstractMergedSortedCacheStoreIterator.java @@ -0,0 +1,166 @@ +/** + * 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 org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.NoSuchElementException; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + * @param + */ +abstract class AbstractMergedSortedCacheStoreIterator implements KeyValueIterator { + private final PeekingKeyValueIterator cacheIterator; + private final KeyValueIterator storeIterator; + protected final StateSerdes serdes; + + AbstractMergedSortedCacheStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + this.cacheIterator = cacheIterator; + this.storeIterator = storeIterator; + this.serdes = serdes; + } + + abstract int compare(final Bytes cacheKey, final KS storeKey); + + abstract K deserializeStoreKey(final KS key); + + abstract KeyValue deserializeStorePair(final KeyValue pair); + + abstract K deserializeCacheKey(final Bytes cacheKey); + + private boolean isDeletedCacheEntry(final KeyValue nextFromCache) { + return nextFromCache.value.value == null; + } + + @Override + public boolean hasNext() { + // skip over items deleted from cache, and corresponding store items if they have the same key + while (cacheIterator.hasNext() && isDeletedCacheEntry(cacheIterator.peekNext())) { + if (storeIterator.hasNext()) { + final KS nextStoreKey = storeIterator.peekNextKey(); + // advance the store iterator if the key is the same as the deleted cache key + if (compare(cacheIterator.peekNextKey(), nextStoreKey) == 0) { + storeIterator.next(); + } + } + cacheIterator.next(); + } + + return cacheIterator.hasNext() || storeIterator.hasNext(); + } + + @Override + public KeyValue next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final Bytes nextCacheKey = cacheIterator.hasNext() ? cacheIterator.peekNextKey() : null; + final KS nextStoreKey = storeIterator.hasNext() ? storeIterator.peekNextKey() : null; + + if (nextCacheKey == null) { + return nextStoreValue(nextStoreKey); + } + + if (nextStoreKey == null) { + return nextCacheValue(nextCacheKey); + } + + final int comparison = compare(nextCacheKey, nextStoreKey); + if (comparison > 0) { + return nextStoreValue(nextStoreKey); + } else if (comparison < 0) { + return nextCacheValue(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return nextCacheValue(nextCacheKey); + } + } + + private KeyValue nextStoreValue(KS nextStoreKey) { + final KeyValue next = storeIterator.next(); + + if (!next.key.equals(nextStoreKey)) { + throw new IllegalStateException("Next record key is not the peeked key value; this should not happen"); + } + + return deserializeStorePair(next); + } + + private KeyValue nextCacheValue(Bytes nextCacheKey) { + final KeyValue next = cacheIterator.next(); + + if (!next.key.equals(nextCacheKey)) { + throw new IllegalStateException("Next record key is not the peeked key value; this should not happen"); + } + + return KeyValue.pair(deserializeCacheKey(next.key), serdes.valueFrom(next.value.value)); + } + + @Override + public K peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + + final Bytes nextCacheKey = cacheIterator.hasNext() ? cacheIterator.peekNextKey() : null; + final KS nextStoreKey = storeIterator.hasNext() ? storeIterator.peekNextKey() : null; + + if (nextCacheKey == null) { + return deserializeStoreKey(nextStoreKey); + } + + if (nextStoreKey == null) { + return serdes.keyFrom(nextCacheKey.get()); + } + + final int comparison = compare(nextCacheKey, nextStoreKey); + if (comparison > 0) { + return deserializeStoreKey(nextStoreKey); + } else if (comparison < 0) { + return deserializeCacheKey(nextCacheKey); + } else { + // skip the same keyed element + storeIterator.next(); + return deserializeCacheKey(nextCacheKey); + } + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove() is not supported"); + } + + @Override + public void close() { + cacheIterator.close(); + storeIterator.close(); + } +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index fdb03fd13e0a5..f9f3077b18d34 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.RecordContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; @@ -31,13 +32,13 @@ import java.util.List; -class CachingKeyValueStore implements KeyValueStore, CachedStateStore { +class CachingKeyValueStore implements WrappedStateStore, KeyValueStore, CachedStateStore { private final KeyValueStore underlying; private final Serde keySerde; private final Serde valueSerde; private CacheFlushListener flushListener; - private String name; + private String cacheName; private ThreadCache cache; private InternalProcessorContext context; private StateSerdes serdes; @@ -69,13 +70,13 @@ public void init(final ProcessorContext context, final StateStore root) { @SuppressWarnings("unchecked") void initInternal(final ProcessorContext context) { this.context = (InternalProcessorContext) context; - this.serdes = new StateSerdes<>(underlying.name(), + this.serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), underlying.name()), keySerde == null ? (Serde) context.keySerde() : keySerde, valueSerde == null ? (Serde) context.valueSerde() : valueSerde); - this.name = context.taskId() + "-" + underlying.name(); + this.cacheName = context.taskId() + "-" + underlying.name(); this.cache = this.context.getCache(); - cache.addDirtyEntryFlushListener(name, new ThreadCache.DirtyEntryFlushListener() { + cache.addDirtyEntryFlushListener(cacheName, new ThreadCache.DirtyEntryFlushListener() { @Override public void apply(final List entries) { for (ThreadCache.DirtyEntry entry : entries) { @@ -108,7 +109,7 @@ public void setFlushListener(final CacheFlushListener flushListener) { @Override public synchronized void flush() { - cache.flush(name); + cache.flush(cacheName); underlying.flush(); } @@ -116,7 +117,7 @@ public synchronized void flush() { public void close() { flush(); underlying.close(); - cache.close(name); + cache.close(cacheName); } @Override @@ -141,12 +142,12 @@ public synchronized V get(final K key) { private void validateStoreOpen() { if (!isOpen()) { - throw new InvalidStateStoreException("Store " + this.name + " is currently closed"); + throw new InvalidStateStoreException("Store " + this.name() + " is currently closed"); } } private V get(final byte[] rawKey) { - final LRUCacheEntry entry = cache.get(name, rawKey); + final LRUCacheEntry entry = cache.get(cacheName, rawKey); if (entry == null) { final byte[] rawValue = underlying.get(Bytes.wrap(rawKey)); if (rawValue == null) { @@ -155,7 +156,7 @@ private V get(final byte[] rawKey) { // only update the cache if this call is on the streamThread // as we don't want other threads to trigger an eviction/flush if (Thread.currentThread().equals(streamThread)) { - cache.put(name, rawKey, new LRUCacheEntry(rawValue)); + cache.put(cacheName, rawKey, new LRUCacheEntry(rawValue)); } return serdes.valueFrom(rawValue); } @@ -173,15 +174,15 @@ public KeyValueIterator range(final K from, final K to) { final byte[] origFrom = serdes.rawKey(from); final byte[] origTo = serdes.rawKey(to); final KeyValueIterator storeIterator = underlying.range(Bytes.wrap(origFrom), Bytes.wrap(origTo)); - final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(name, origFrom, origTo); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(cacheName, origFrom, origTo); return new MergedSortedCacheKeyValueStoreIterator<>(cacheIterator, storeIterator, serdes); } @Override public KeyValueIterator all() { validateStoreOpen(); - final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>(name, underlying.all()); - final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(name); + final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>(this.name(), underlying.all()); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.all(cacheName); return new MergedSortedCacheKeyValueStoreIterator<>(cacheIterator, storeIterator, serdes); } @@ -199,7 +200,7 @@ public synchronized void put(final K key, final V value) { private synchronized void put(final byte[] rawKey, final V value) { final byte[] rawValue = serdes.rawValue(value); - cache.put(name, rawKey, new LRUCacheEntry(rawValue, true, context.offset(), + cache.put(cacheName, rawKey, new LRUCacheEntry(rawValue, true, context.offset(), context.timestamp(), context.partition(), context.topic())); } @@ -226,7 +227,7 @@ public synchronized V delete(final K key) { validateStoreOpen(); final byte[] rawKey = serdes.rawKey(key); final V v = get(rawKey); - cache.delete(name, serdes.rawKey(key)); + cache.delete(cacheName, serdes.rawKey(key)); underlying.delete(Bytes.wrap(rawKey)); return v; } @@ -234,4 +235,12 @@ public synchronized V delete(final K key) { KeyValueStore underlying() { return underlying; } + + @Override + public StateStore inner() { + if (underlying instanceof WrappedStateStore) { + return ((WrappedStateStore) underlying).inner(); + } + return underlying; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java index 17c4ee0084684..ed64246b18d09 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingSessionStore.java @@ -18,38 +18,38 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.kstream.internals.SessionKeySerde; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.RecordContext; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StateSerdes; import java.util.List; -import java.util.NoSuchElementException; -class CachingSessionStore implements SessionStore, CachedStateStore, AGG> { +class CachingSessionStore extends WrappedStateStore.AbstractWrappedStateStore implements SessionStore, CachedStateStore, AGG> { - private final SegmentedBytesStore bytesStore; + private final SessionStore bytesStore; private final SessionKeySchema keySchema; private Serde keySerde; private final Serde aggSerde; private InternalProcessorContext context; - private String name; - private StateSerdes, AGG> serdes; + private String cacheName; + private StateSerdes serdes; private ThreadCache cache; private CacheFlushListener, AGG> flushListener; + private String topic; - CachingSessionStore(final SegmentedBytesStore bytesStore, + CachingSessionStore(final SessionStore bytesStore, final Serde keySerde, final Serde aggSerde) { + super(bytesStore); this.bytesStore = bytesStore; this.keySerde = keySerde; this.aggSerde = aggSerde; @@ -60,17 +60,17 @@ public KeyValueIterator, AGG> findSessions(final K key, final long earliestSessionEndTime, final long latestSessionStartTime) { validateStoreOpen(); - final Bytes binarySessionId = Bytes.wrap(keySerde.serializer().serialize(name, key)); - final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(name, + final Bytes binarySessionId = Bytes.wrap(keySerde.serializer().serialize(topic, key)); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(cacheName, keySchema.lowerRange(binarySessionId, earliestSessionEndTime).get(), keySchema.upperRange(binarySessionId, latestSessionStartTime).get()); - final KeyValueIterator storeIterator = bytesStore.fetch(binarySessionId, earliestSessionEndTime, latestSessionStartTime); + final KeyValueIterator, byte[]> storeIterator = bytesStore.findSessions(binarySessionId, earliestSessionEndTime, latestSessionStartTime); final HasNextCondition hasNextCondition = keySchema.hasNextCondition(binarySessionId, - earliestSessionEndTime, - latestSessionStartTime); + earliestSessionEndTime, + latestSessionStartTime); final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition); - return new MergedSortedCacheKeyValueStoreIterator<>(filteredCacheIterator, storeIterator, serdes); + return new MergedSortedCacheSessionStoreIterator<>(filteredCacheIterator, storeIterator, serdes); } @@ -81,10 +81,10 @@ public void remove(final Windowed sessionKey) { public void put(final Windowed key, AGG value) { validateStoreOpen(); - final Bytes binaryKey = SessionKeySerde.toBinary(key, keySerde.serializer()); + final Bytes binaryKey = SessionKeySerde.toBinary(key, keySerde.serializer(), topic); final LRUCacheEntry entry = new LRUCacheEntry(serdes.rawValue(value), true, context.offset(), key.window().end(), context.partition(), context.topic()); - cache.put(name, binaryKey.get(), entry); + cache.put(cacheName, binaryKey.get(), entry); } @Override @@ -92,13 +92,9 @@ public KeyValueIterator, AGG> fetch(final K key) { return findSessions(key, 0, Long.MAX_VALUE); } - - public String name() { - return bytesStore.name(); - } - @SuppressWarnings("unchecked") public void init(final ProcessorContext context, final StateStore root) { + topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), root.name()); bytesStore.init(context, root); initInternal((InternalProcessorContext) context); } @@ -107,19 +103,15 @@ public void init(final ProcessorContext context, final StateStore root) { private void initInternal(final InternalProcessorContext context) { this.context = context; - if (keySerde == null) { - keySerde = (Serde) context.keySerde(); - } - + keySchema.init(topic); + this.serdes = new StateSerdes<>(topic, + keySerde == null ? (Serde) context.keySerde() : keySerde, + aggSerde == null ? (Serde) context.valueSerde() : aggSerde); - this.serdes = (StateSerdes, AGG>) new StateSerdes<>(bytesStore.name(), - new SessionKeySerde<>(keySerde), - aggSerde == null ? context.valueSerde() : aggSerde); - - this.name = context.taskId() + "-" + bytesStore.name(); + this.cacheName = context.taskId() + "-" + bytesStore.name(); this.cache = this.context.getCache(); - cache.addDirtyEntryFlushListener(name, new ThreadCache.DirtyEntryFlushListener() { + cache.addDirtyEntryFlushListener(cacheName, new ThreadCache.DirtyEntryFlushListener() { @Override public void apply(final List entries) { for (ThreadCache.DirtyEntry entry : entries) { @@ -135,108 +127,43 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern final RecordContext current = context.recordContext(); context.setRecordContext(entry.recordContext()); try { + final Windowed key = SessionKeySerde.from(binaryKey.get(), keySerde.deserializer(), topic); if (flushListener != null) { - final Windowed key = SessionKeySerde.from(binaryKey.get(), keySerde.deserializer()); final AGG newValue = serdes.valueFrom(entry.newValue()); final AGG oldValue = fetchPrevious(binaryKey); if (!(newValue == null && oldValue == null)) { - flushListener.apply(key, newValue == null ? null : newValue, oldValue); + flushListener.apply(key, newValue, oldValue); } - } - bytesStore.put(binaryKey, entry.newValue()); + bytesStore.put(new Windowed<>(Bytes.wrap(serdes.rawKey(key.key())), key.window()), entry.newValue()); } finally { context.setRecordContext(current); } } private AGG fetchPrevious(final Bytes key) { - final byte[] bytes = bytesStore.get(key); - if (bytes == null) { - return null; + try (final KeyValueIterator, byte[]> iterator = bytesStore.fetch(key)) { + if (!iterator.hasNext()) { + return null; + } + return serdes.valueFrom(iterator.next().value); } - return serdes.valueFrom(bytes); } public void flush() { - cache.flush(name); + cache.flush(cacheName); bytesStore.flush(); } public void close() { flush(); bytesStore.close(); - cache.close(name); - } - - public boolean persistent() { - return bytesStore.persistent(); - } - - public boolean isOpen() { - return bytesStore.isOpen(); + cache.close(cacheName); } public void setFlushListener(CacheFlushListener, AGG> flushListener) { this.flushListener = flushListener; } - private void validateStoreOpen() { - if (!isOpen()) { - throw new InvalidStateStoreException("Store " + this.name + " is currently closed"); - } - } - - - private static class FilteredCacheIterator implements PeekingKeyValueIterator { - private final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator; - private final HasNextCondition hasNextCondition; - - FilteredCacheIterator(final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator, - final HasNextCondition hasNextCondition) { - this.cacheIterator = cacheIterator; - this.hasNextCondition = hasNextCondition; - } - - @Override - public void close() { - // no-op - } - - @Override - public Bytes peekNextKey() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - return cacheIterator.peekNextKey(); - } - - @Override - public boolean hasNext() { - return hasNextCondition.hasNext(cacheIterator); - } - - @Override - public KeyValue next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - return cacheIterator.next(); - - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - - @Override - public KeyValue peekNext() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - return cacheIterator.peekNext(); - } - } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java index bd252f147d0ae..37ce336f0326d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingWindowStore.java @@ -17,60 +17,59 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.internals.CacheFlushListener; import org.apache.kafka.streams.kstream.internals.TimeWindow; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.InternalProcessorContext; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.RecordContext; -import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import java.util.List; -class CachingWindowStore implements WindowStore, CachedStateStore, V> { +class CachingWindowStore extends WrappedStateStore.AbstractWrappedStateStore implements WindowStore, CachedStateStore, V> { - private final SegmentedBytesStore underlying; + private final WindowStore underlying; private final Serde keySerde; private final Serde valueSerde; private CacheFlushListener, V> flushListener; private final long windowSize; + private final SegmentedBytesStore.KeySchema keySchema = new WindowStoreKeySchema(); + private String name; private ThreadCache cache; private InternalProcessorContext context; private StateSerdes serdes; - CachingWindowStore(final SegmentedBytesStore underlying, + CachingWindowStore(final WindowStore underlying, final Serde keySerde, final Serde valueSerde, final long windowSize) { + super(underlying); this.underlying = underlying; this.keySerde = keySerde; this.valueSerde = valueSerde; this.windowSize = windowSize; } - @Override - public String name() { - return underlying.name(); - } - @SuppressWarnings("unchecked") @Override public void init(final ProcessorContext context, final StateStore root) { underlying.init(context, root); initInternal(context); + keySchema.init(context.applicationId()); } @SuppressWarnings("unchecked") void initInternal(final ProcessorContext context) { this.context = (InternalProcessorContext) context; - this.serdes = new StateSerdes<>(underlying.name(), + this.serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), underlying.name()), keySerde == null ? (Serde) context.keySerde() : keySerde, valueSerde == null ? (Serde) context.valueSerde() : valueSerde); @@ -80,13 +79,14 @@ void initInternal(final ProcessorContext context) { @Override public void apply(final List entries) { for (ThreadCache.DirtyEntry entry : entries) { - final byte[] binaryKey = entry.key().get(); - final Bytes key = WindowStoreUtils.bytesKeyFromBinaryKey(binaryKey); - final long timestamp = WindowStoreUtils.timestampFromBinaryKey(binaryKey); - final Windowed windowedKey = new Windowed<>(WindowStoreUtils.keyFromBinaryKey(binaryKey, serdes), + final byte[] binaryWindowKey = entry.key().get(); + final long timestamp = WindowStoreUtils.timestampFromBinaryKey(binaryWindowKey); + + final Windowed windowedKey = new Windowed<>(WindowStoreUtils.keyFromBinaryKey(binaryWindowKey, serdes), new TimeWindow(timestamp, timestamp + windowSize)); - maybeForward(entry, Bytes.wrap(binaryKey), windowedKey, (InternalProcessorContext) context); - underlying.put(Bytes.wrap(WindowStoreUtils.toBinaryKey(key, timestamp, 0, WindowStoreUtils.INNER_SERDES)), entry.newValue()); + final Bytes key = WindowStoreUtils.bytesKeyFromBinaryKey(binaryWindowKey); + maybeForward(entry, key, windowedKey, (InternalProcessorContext) context); + underlying.put(key, entry.newValue(), timestamp); } } }); @@ -102,7 +102,7 @@ private void maybeForward(final ThreadCache.DirtyEntry entry, context.setRecordContext(entry.recordContext()); try { flushListener.apply(windowedKey, - serdes.valueFrom(entry.newValue()), fetchPrevious(key)); + serdes.valueFrom(entry.newValue()), fetchPrevious(key, windowedKey.window().start())); } finally { context.setRecordContext(current); } @@ -127,16 +127,6 @@ public void close() { cache.close(name); } - @Override - public boolean persistent() { - return underlying.persistent(); - } - - @Override - public boolean isOpen() { - return underlying.isOpen(); - } - @Override public synchronized void put(final K key, final V value) { put(key, value, context.timestamp()); @@ -158,23 +148,28 @@ public synchronized WindowStoreIterator fetch(final K key, final long timeFro byte[] binaryFrom = WindowStoreUtils.toBinaryKey(key, timeFrom, 0, serdes); byte[] binaryTo = WindowStoreUtils.toBinaryKey(key, timeTo, 0, serdes); - final KeyValueIterator underlyingIterator = underlying.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); + final WindowStoreIterator underlyingIterator = underlying.fetch(Bytes.wrap(serdes.rawKey(key)), timeFrom, timeTo); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(name, binaryFrom, binaryTo); - return new MergedSortedCachedWindowStoreIterator<>(cacheIterator, new DelegatingPeekingKeyValueIterator<>(name, underlyingIterator), serdes); - } - private V fetchPrevious(final Bytes key) { - final byte[] result = underlying.get(key); - if (result == null) { - return null; - } - return serdes.valueFrom(result); + final HasNextCondition hasNextCondition = keySchema.hasNextCondition(Bytes.wrap(serdes.rawKey(key)), + timeFrom, + timeTo); + final PeekingKeyValueIterator filteredCacheIterator = new FilteredCacheIterator(cacheIterator, hasNextCondition); + + return new MergedSortedCacheWindowStoreIterator<>(filteredCacheIterator, + underlyingIterator, + new StateSerdes<>(serdes.topic(), Serdes.Long(), serdes.valueSerde())); } - private void validateStoreOpen() { - if (!isOpen()) { - throw new InvalidStateStoreException("Store " + this.name + " is currently closed"); + + private V fetchPrevious(final Bytes key, final long timestamp) { + try (final WindowStoreIterator iter = underlying.fetch(key, timestamp, timestamp)) { + if (!iter.hasNext()) { + return null; + } else { + return serdes.valueFrom(iter.next().value); + } } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java new file mode 100644 index 0000000000000..9ece123df6d71 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStore.java @@ -0,0 +1,100 @@ +/** + * 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 org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.List; + +public class ChangeLoggingKeyValueBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { + private final KeyValueStore inner; + private StoreChangeLogger changeLogger; + + public ChangeLoggingKeyValueBytesStore(final KeyValueStore inner) { + super(inner); + this.inner = inner; + } + + @Override + public void init(final ProcessorContext context, final StateStore root) { + inner.init(context, root); + this.changeLogger = new StoreChangeLogger<>( + inner.name(), + context, + WindowStoreUtils.getInnerStateSerde( + ProcessorStateManager.storeChangelogTopic( + context.applicationId(), + inner.name()))); + } + + + @Override + public void put(final Bytes key, final byte[] value) { + inner.put(key, value); + changeLogger.logChange(key, value); + } + + @Override + public byte[] putIfAbsent(final Bytes key, final byte[] value) { + final byte[] previous = get(key); + if (previous == null) { + put(key, value); + } + return previous; + } + + @Override + public void putAll(final List> entries) { + inner.putAll(entries); + for (KeyValue entry : entries) { + changeLogger.logChange(entry.key, entry.value); + } + } + + @Override + public byte[] delete(final Bytes key) { + final byte[] oldValue = inner.get(key); + put(key, null); + return oldValue; + } + + @Override + public byte[] get(final Bytes key) { + return inner.get(key); + } + + @Override + public KeyValueIterator range(final Bytes from, final Bytes to) { + return inner.range(from, to); + } + + @Override + public KeyValueIterator all() { + return inner.all(); + } + + @Override + public long approximateNumEntries() { + return inner.approximateNumEntries(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java new file mode 100644 index 0000000000000..d50f90764a6c6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStore.java @@ -0,0 +1,123 @@ +/** + * 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 org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.ArrayList; +import java.util.List; + +class ChangeLoggingKeyValueStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { + private final ChangeLoggingKeyValueBytesStore innerBytes; + private final Serde keySerde; + private final Serde valueSerde; + private StateSerdes serdes; + + ChangeLoggingKeyValueStore(final KeyValueStore bytesStore, + final Serde keySerde, + final Serde valueSerde) { + this(new ChangeLoggingKeyValueBytesStore(bytesStore), keySerde, valueSerde); + } + + private ChangeLoggingKeyValueStore(final ChangeLoggingKeyValueBytesStore bytesStore, + final Serde keySerde, + final Serde valueSerde) { + super(bytesStore); + this.innerBytes = bytesStore; + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + @SuppressWarnings("unchecked") + @Override + public void init(final ProcessorContext context, final StateStore root) { + innerBytes.init(context, root); + + this.serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), innerBytes.name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + } + + @Override + public void put(final K key, final V value) { + final Bytes bytesKey = Bytes.wrap(serdes.rawKey(key)); + final byte[] bytesValue = serdes.rawValue(value); + innerBytes.put(bytesKey, bytesValue); + } + + @Override + public V putIfAbsent(final K key, final V value) { + final V v = get(key); + if (v == null) { + put(key, value); + } + return v; + } + + @Override + public void putAll(final List> entries) { + final List> keyValues = new ArrayList<>(); + for (final KeyValue entry : entries) { + keyValues.add(KeyValue.pair(Bytes.wrap(serdes.rawKey(entry.key)), serdes.rawValue(entry.value))); + } + innerBytes.putAll(keyValues); + } + + @Override + public V delete(final K key) { + final byte[] oldValue = innerBytes.delete(Bytes.wrap(serdes.rawKey(key))); + if (oldValue == null) { + return null; + } + return serdes.valueFrom(oldValue); + } + + @Override + public V get(final K key) { + final byte[] rawValue = innerBytes.get(Bytes.wrap(serdes.rawKey(key))); + if (rawValue == null) { + return null; + } + return serdes.valueFrom(rawValue); + } + + @Override + public KeyValueIterator range(final K from, final K to) { + return new SerializedKeyValueIterator<>(innerBytes.range(Bytes.wrap(serdes.rawKey(from)), + Bytes.wrap(serdes.rawKey(to))), + serdes); + } + + @Override + public KeyValueIterator all() { + return new SerializedKeyValueIterator<>(innerBytes.all(), serdes); + } + + @Override + public long approximateNumEntries() { + return innerBytes.approximateNumEntries(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java index 14b8f1710e287..2a5a1a19f433b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStore.java @@ -19,19 +19,21 @@ import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; /** * Simple wrapper around a {@link SegmentedBytesStore} to support writing * updates to a changelog */ -class ChangeLoggingSegmentedBytesStore implements SegmentedBytesStore { +class ChangeLoggingSegmentedBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements SegmentedBytesStore { private final SegmentedBytesStore bytesStore; private StoreChangeLogger changeLogger; ChangeLoggingSegmentedBytesStore(final SegmentedBytesStore bytesStore) { + super(bytesStore); this.bytesStore = bytesStore; } @@ -60,36 +62,17 @@ public byte[] get(final Bytes key) { return bytesStore.get(key); } - @Override - public String name() { - return bytesStore.name(); - } @Override @SuppressWarnings("unchecked") public void init(final ProcessorContext context, final StateStore root) { bytesStore.init(context, root); - changeLogger = new StoreChangeLogger<>(name(), context, WindowStoreUtils.INNER_SERDES); - } - - @Override - public void flush() { - bytesStore.flush(); - } - - @Override - public void close() { - bytesStore.close(); + changeLogger = new StoreChangeLogger<>( + name(), + context, + WindowStoreUtils.getInnerStateSerde( + ProcessorStateManager.storeChangelogTopic( + context.applicationId(), + name()))); } - - @Override - public boolean persistent() { - return bytesStore.persistent(); - } - - @Override - public boolean isOpen() { - return bytesStore.isOpen(); - } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java index b33c0f0ceab3d..e0f1ec85b037b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyWindowStore.java @@ -61,6 +61,11 @@ public WindowStoreIterator fetch(final K key, final long timeFrom, final long public void close() { } + @Override + public Long peekNextKey() { + throw new NoSuchElementException(); + } + @Override public boolean hasNext() { return false; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java index eb57acee688cf..f3101b17f9c54 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIterator.java @@ -22,7 +22,7 @@ import java.util.NoSuchElementException; -public class DelegatingPeekingKeyValueIterator implements KeyValueIterator { +public class DelegatingPeekingKeyValueIterator implements KeyValueIterator, PeekingKeyValueIterator { private final String storeName; private final KeyValueIterator underlying; private KeyValue next; @@ -78,4 +78,12 @@ public synchronized KeyValue next() { public void remove() { throw new UnsupportedOperationException("remove not supported"); } + + @Override + public KeyValue peekNext() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return next; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.java new file mode 100644 index 0000000000000..135190325c854 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/FilteredCacheIterator.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.streams.state.internals; + +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; + +import java.util.NoSuchElementException; + +class FilteredCacheIterator implements PeekingKeyValueIterator { + private final PeekingKeyValueIterator cacheIterator; + private final HasNextCondition hasNextCondition; + + FilteredCacheIterator(final PeekingKeyValueIterator cacheIterator, + final HasNextCondition hasNextCondition) { + this.cacheIterator = cacheIterator; + this.hasNextCondition = hasNextCondition; + } + + @Override + public void close() { + // no-op + } + + @Override + public Bytes peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return cacheIterator.peekNextKey(); + } + + @Override + public boolean hasNext() { + return hasNextCondition.hasNext(cacheIterator); + } + + @Override + public KeyValue next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return cacheIterator.next(); + + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + @Override + public KeyValue peekNext() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return cacheIterator.peekNext(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java index d81f6fbac2409..f402390159743 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; @@ -35,7 +36,6 @@ public class InMemoryKeyValueLoggedStore implements KeyValueStore { private final String storeName; private StoreChangeLogger changeLogger; - private ProcessorContext context; public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, Serde keySerde, Serde valueSerde) { this.storeName = storeName; @@ -52,13 +52,13 @@ public String name() { @Override @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { - this.context = context; inner.init(context, root); // construct the serde - StateSerdes serdes = new StateSerdes<>(storeName, - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + StateSerdes serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), inner.name()), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); this.changeLogger = new StoreChangeLogger<>(storeName, context, serdes); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.java new file mode 100644 index 0000000000000..dbcc219a80dc8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStore.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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateRestoreCallback; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + + +public class InMemoryKeyValueStore implements KeyValueStore { + private final String name; + private final Serde keySerde; + private final Serde valueSerde; + private final NavigableMap map; + private volatile boolean open = false; + + private StateSerdes serdes; + + public InMemoryKeyValueStore(final String name, final Serde keySerde, final Serde valueSerde) { + this.name = name; + this.keySerde = keySerde; + this.valueSerde = valueSerde; + + // TODO: when we have serde associated with class types, we can + // improve this situation by passing the comparator here. + this.map = new TreeMap<>(); + } + + public KeyValueStore enableLogging() { + return new InMemoryKeyValueLoggedStore<>(name, this, keySerde, valueSerde); + } + + @Override + public String name() { + return this.name; + } + + @Override + @SuppressWarnings("unchecked") + public void init(ProcessorContext context, StateStore root) { + // construct the serde + this.serdes = new StateSerdes<>(name, + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + + if (root != null) { + // register the store + context.register(root, true, new StateRestoreCallback() { + @Override + public void restore(byte[] key, byte[] value) { + // check value for null, to avoid deserialization error. + if (value == null) { + put(serdes.keyFrom(key), null); + } else { + put(serdes.keyFrom(key), serdes.valueFrom(value)); + } + } + }); + } + + this.open = true; + } + + @Override + public boolean persistent() { + return false; + } + + @Override + public boolean isOpen() { + return this.open; + } + + @Override + public synchronized V get(K key) { + return this.map.get(key); + } + + @Override + public synchronized void put(K key, V value) { + this.map.put(key, value); + } + + @Override + public synchronized V putIfAbsent(K key, V value) { + V originalValue = get(key); + if (originalValue == null) { + put(key, value); + } + return originalValue; + } + + @Override + public synchronized void putAll(List> entries) { + for (KeyValue entry : entries) + put(entry.key, entry.value); + } + + @Override + public synchronized V delete(K key) { + return this.map.remove(key); + } + + @Override + public synchronized KeyValueIterator range(K from, K to) { + return new DelegatingPeekingKeyValueIterator<>(name, new InMemoryKeyValueIterator<>(this.map.subMap(from, true, to, true).entrySet().iterator())); + } + + @Override + public synchronized KeyValueIterator all() { + final TreeMap copy = new TreeMap<>(this.map); + return new DelegatingPeekingKeyValueIterator<>(name, new InMemoryKeyValueIterator<>(copy.entrySet().iterator())); + } + + @Override + public long approximateNumEntries() { + return this.map.size(); + } + + @Override + public void flush() { + // do-nothing since it is in-memory + } + + @Override + public void close() { + this.map.clear(); + this.open = false; + } + + private static class InMemoryKeyValueIterator implements KeyValueIterator { + private final Iterator> iter; + + private InMemoryKeyValueIterator(Iterator> iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public KeyValue next() { + Map.Entry entry = iter.next(); + return new KeyValue<>(entry.getKey(), entry.getValue()); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + // do nothing + } + + @Override + public K peekNextKey() { + throw new UnsupportedOperationException("peekNextKey() not supported in " + getClass().getName()); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index e00f8ab4f3be6..05130ef34d74a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; @@ -94,9 +95,10 @@ public String name() { @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { // construct the serde - this.serdes = new StateSerdes<>(name, - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + this.serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); // register the store context.register(root, true, new StateRestoreCallback() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index 4e1f40e52dfef..4ab6fabc1912c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -21,6 +21,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; @@ -103,9 +104,10 @@ public String name() { @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { // construct the serde - this.serdes = new StateSerdes<>(name, - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + this.serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); // register the store context.register(root, true, new StateRestoreCallback() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java index c9a6866d9d17a..b860e161e0d9f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheKeyValueStoreIterator.java @@ -21,141 +21,37 @@ import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StateSerdes; -import java.util.Comparator; -import java.util.NoSuchElementException; - /** * Merges two iterators. Assumes each of them is sorted by key * * @param * @param */ -class MergedSortedCacheKeyValueStoreIterator implements KeyValueIterator { - private final PeekingKeyValueIterator cacheIterator; - private final KeyValueIterator storeIterator; - private final StateSerdes serdes; - private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; - - public MergedSortedCacheKeyValueStoreIterator(final PeekingKeyValueIterator cacheIterator, - final KeyValueIterator storeIterator, - final StateSerdes serdes) { - this.cacheIterator = cacheIterator; - this.storeIterator = storeIterator; - this.serdes = serdes; - } - - @Override - public boolean hasNext() { - while (cacheIterator.hasNext() && isDeletedCacheEntry(cacheIterator.peekNext())) { - if (storeIterator.hasNext()) { - final Bytes storeKey = storeIterator.peekNextKey(); - // advance the store iterator if the key is the same as the deleted cache key - if (storeKey.equals(cacheIterator.peekNextKey())) { - storeIterator.next(); - } - } - // skip over items deleted from cache - cacheIterator.next(); - } - return cacheIterator.hasNext() || storeIterator.hasNext(); - } +class MergedSortedCacheKeyValueStoreIterator extends AbstractMergedSortedCacheStoreIterator { - - private boolean isDeletedCacheEntry(final KeyValue nextFromCache) { - return nextFromCache.value.value == null; + MergedSortedCacheKeyValueStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, serdes); } - @Override - public KeyValue next() { - - return internalNext(new NextValueFunction>() { - @Override - public KeyValue apply(final byte[] cacheKey, final byte[] storeKey) { - if (cacheKey == null) { - return nextStoreValue(); - } - - if (storeKey == null) { - return nextCacheValue(); - } - - final int comparison = comparator.compare(cacheKey, storeKey); - if (comparison > 0) { - return nextStoreValue(); - } else if (comparison < 0) { - return nextCacheValue(); - } else { - storeIterator.next(); - return nextCacheValue(); - } - } - }); + public KeyValue deserializeStorePair(KeyValue pair) { + return KeyValue.pair(serdes.keyFrom(pair.key.get()), serdes.valueFrom(pair.value)); } @Override - public K peekNextKey() { - return internalNext(new NextValueFunction() { - @Override - public K apply(final byte[] cacheKey, final byte[] storeKey) { - if (cacheKey == null) { - return serdes.keyFrom(storeKey); - } - - if (storeKey == null) { - return serdes.keyFrom(cacheKey); - } - - final int comparison = comparator.compare(cacheKey, storeKey); - if (comparison > 0) { - return serdes.keyFrom(storeKey); - } else { - return serdes.keyFrom(cacheKey); - } - } - }); - } - - interface NextValueFunction { - T apply(final byte[] cacheKey, final byte [] storeKey); - } - - private T internalNext(final NextValueFunction nextValueFunction) { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - byte[] nextCacheKey = null; - if (cacheIterator.hasNext()) { - nextCacheKey = cacheIterator.peekNextKey().get(); - } - - byte[] nextStoreKey = null; - if (storeIterator.hasNext()) { - nextStoreKey = storeIterator.peekNextKey().get(); - } - - return nextValueFunction.apply(nextCacheKey, nextStoreKey); - } - - private KeyValue nextCacheValue() { - final KeyValue next = cacheIterator.next(); - return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value.value)); - } - - private KeyValue nextStoreValue() { - final KeyValue next = storeIterator.next(); - return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value)); + K deserializeCacheKey(final Bytes cacheKey) { + return serdes.keyFrom(cacheKey.get()); } @Override - public void remove() { - throw new UnsupportedOperationException("remove not supported"); + public K deserializeStoreKey(Bytes key) { + return serdes.keyFrom(key.get()); } @Override - public void close() { - cacheIterator.close(); - storeIterator.close(); + public int compare(Bytes cacheKey, Bytes storeKey) { + return cacheKey.compareTo(storeKey); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java new file mode 100644 index 0000000000000..8ecc65468aa63 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIterator.java @@ -0,0 +1,71 @@ +/** + * 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 org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionKeySerde; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + * @param + */ +class MergedSortedCacheSessionStoreIterator extends AbstractMergedSortedCacheStoreIterator, Windowed, AGG> { + private final StateSerdes rawSerdes; + + + MergedSortedCacheSessionStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator, byte[]> storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, new StateSerdes<>(serdes.topic(), + new SessionKeySerde<>(serdes.keySerde()), + serdes.valueSerde())); + + rawSerdes = serdes; + } + + @Override + public KeyValue, AGG> deserializeStorePair(KeyValue, byte[]> pair) { + final K key = rawSerdes.keyFrom(pair.key.key().get()); + return KeyValue.pair(new Windowed<>(key, pair.key.window()), serdes.valueFrom(pair.value)); + } + + @Override + Windowed deserializeCacheKey(final Bytes cacheKey) { + return SessionKeySerde.from(cacheKey.get(), rawSerdes.keyDeserializer(), rawSerdes.topic()); + } + + @Override + public Windowed deserializeStoreKey(Windowed key) { + final K originalKey = rawSerdes.keyFrom(key.key().get()); + return new Windowed(originalKey, key.window()); + } + + @Override + public int compare(Bytes cacheKey, Windowed storeKey) { + Bytes storeKeyBytes = SessionKeySerde.bytesToBinary(storeKey); + return cacheKey.compareTo(storeKeyBytes); + } +} + diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java new file mode 100644 index 0000000000000..a9d097320086c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIterator.java @@ -0,0 +1,58 @@ +/** + * 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 org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.WindowStoreIterator; + +/** + * Merges two iterators. Assumes each of them is sorted by key + * + * @param + */ +class MergedSortedCacheWindowStoreIterator extends AbstractMergedSortedCacheStoreIterator implements WindowStoreIterator { + + MergedSortedCacheWindowStoreIterator(final PeekingKeyValueIterator cacheIterator, + final KeyValueIterator storeIterator, + final StateSerdes serdes) { + super(cacheIterator, storeIterator, serdes); + } + + @Override + public KeyValue deserializeStorePair(final KeyValue pair) { + return KeyValue.pair(pair.key, serdes.valueFrom(pair.value)); + } + + @Override + Long deserializeCacheKey(final Bytes cacheKey) { + return WindowStoreUtils.timestampFromBinaryKey(cacheKey.get()); + } + + @Override + public Long deserializeStoreKey(final Long key) { + return key; + } + + @Override + public int compare(final Bytes cacheKey, final Long storeKey) { + final Long cacheTimestamp = WindowStoreUtils.timestampFromBinaryKey(cacheKey.get()); + return cacheTimestamp.compareTo(storeKey); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java deleted file mode 100644 index e210e73111fe9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MergedSortedCachedWindowStoreIterator.java +++ /dev/null @@ -1,107 +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.streams.state.internals; - -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.StateSerdes; -import org.apache.kafka.streams.state.WindowStoreIterator; - -import java.util.NoSuchElementException; - -/** - * Merges two iterators. Assumes each of them is sorted by key - * - * @param - * @param - */ -class MergedSortedCachedWindowStoreIterator implements WindowStoreIterator { - private final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator; - private final KeyValueIterator storeIterator; - private final StateSerdes serdes; - - public MergedSortedCachedWindowStoreIterator(final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator, - final KeyValueIterator storeIterator, - final StateSerdes serdes) { - this.cacheIterator = cacheIterator; - this.storeIterator = storeIterator; - this.serdes = serdes; - } - - @Override - public boolean hasNext() { - return cacheIterator.hasNext() || storeIterator.hasNext(); - } - - - @Override - public KeyValue next() { - if (!hasNext()) { - throw new NoSuchElementException(); - } - - Long nextCacheTimestamp = null; - if (cacheIterator.hasNext()) { - nextCacheTimestamp = WindowStoreUtils.timestampFromBinaryKey(cacheIterator.peekNextKey().get()); - } - - Long nextStoreTimestamp = null; - if (storeIterator.hasNext()) { - nextStoreTimestamp = WindowStoreUtils.timestampFromBinaryKey(storeIterator.peekNextKey().get()); - } - - if (nextCacheTimestamp == null) { - return nextStoreValue(nextStoreTimestamp); - } - - if (nextStoreTimestamp == null) { - return nextCacheValue(nextCacheTimestamp); - } - - final int comparison = nextCacheTimestamp.compareTo(nextStoreTimestamp); - if (comparison > 0) { - return nextStoreValue(nextStoreTimestamp); - } else if (comparison < 0) { - return nextCacheValue(nextCacheTimestamp); - } else { - storeIterator.next(); - return nextCacheValue(nextCacheTimestamp); - } - } - - private KeyValue nextCacheValue(final Long timestamp) { - final KeyValue next = cacheIterator.next(); - return KeyValue.pair(timestamp, serdes.valueFrom(next.value.value)); - } - - private KeyValue nextStoreValue(final Long timestamp) { - final KeyValue next = storeIterator.next(); - return KeyValue.pair(timestamp, serdes.valueFrom(next.value)); - } - - @Override - public void remove() { - throw new UnsupportedOperationException("remove not supported"); - } - - @Override - public void close() { - cacheIterator.close(); - storeIterator.close(); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 926e5d4a4acf0..dc6b7a5582b0b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -35,7 +35,7 @@ * @param * @param */ -public class MeteredKeyValueStore implements KeyValueStore { +public class MeteredKeyValueStore extends WrappedStateStore.AbstractWrappedStateStore implements KeyValueStore { protected final KeyValueStore inner; protected final String metricScope; @@ -102,47 +102,35 @@ public void run() { }; // always wrap the store with the metered store - public MeteredKeyValueStore(final KeyValueStore inner, String metricScope, Time time) { + public MeteredKeyValueStore(final KeyValueStore inner, + final String metricScope, + final Time time) { + super(inner); this.inner = inner; this.metricScope = metricScope; this.time = time != null ? time : Time.SYSTEM; } - @Override - public String name() { - return inner.name(); - } - @Override public void init(ProcessorContext context, StateStore root) { final String name = name(); this.context = context; this.root = root; this.metrics = (StreamsMetricsImpl) context.metrics(); - this.putTime = this.metrics.addLatencySensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); - this.putIfAbsentTime = this.metrics.addLatencySensor(metricScope, name, "put-if-absent", Sensor.RecordingLevel.DEBUG); - this.getTime = this.metrics.addLatencySensor(metricScope, name, "get", Sensor.RecordingLevel.DEBUG); - this.deleteTime = this.metrics.addLatencySensor(metricScope, name, "delete", Sensor.RecordingLevel.DEBUG); - this.putAllTime = this.metrics.addLatencySensor(metricScope, name, "put-all", Sensor.RecordingLevel.DEBUG); - this.allTime = this.metrics.addLatencySensor(metricScope, name, "all", Sensor.RecordingLevel.DEBUG); - this.rangeTime = this.metrics.addLatencySensor(metricScope, name, "range", Sensor.RecordingLevel.DEBUG); - this.flushTime = this.metrics.addLatencySensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); - this.restoreTime = this.metrics.addLatencySensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); + this.putTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); + this.putIfAbsentTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "put-if-absent", Sensor.RecordingLevel.DEBUG); + this.getTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "get", Sensor.RecordingLevel.DEBUG); + this.deleteTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "delete", Sensor.RecordingLevel.DEBUG); + this.putAllTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "put-all", Sensor.RecordingLevel.DEBUG); + this.allTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "all", Sensor.RecordingLevel.DEBUG); + this.rangeTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "range", Sensor.RecordingLevel.DEBUG); + this.flushTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); + this.restoreTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); // register and possibly restore the state from the logs metrics.measureLatencyNs(time, initDelegate, this.restoreTime); } - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - @Override public V get(K key) { this.key = key; @@ -193,11 +181,6 @@ public long approximateNumEntries() { return this.inner.approximateNumEntries(); } - @Override - public void close() { - inner.close(); - } - @Override public void flush() { metrics.measureLatencyNs(time, flushDelegate, this.flushTime); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java index e0ed03ed06cd9..4d3771ea55edf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStore.java @@ -27,7 +27,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; -class MeteredSegmentedBytesStore implements SegmentedBytesStore { +class MeteredSegmentedBytesStore extends WrappedStateStore.AbstractWrappedStateStore implements SegmentedBytesStore { private final SegmentedBytesStore inner; private final String metricScope; @@ -40,28 +40,26 @@ class MeteredSegmentedBytesStore implements SegmentedBytesStore { private Sensor getTime; private Sensor removeTime; - MeteredSegmentedBytesStore(final SegmentedBytesStore inner, String metricScope, Time time) { + MeteredSegmentedBytesStore(final SegmentedBytesStore inner, + final String metricScope, + final Time time) { + super(inner); this.inner = inner; this.metricScope = metricScope; this.time = time != null ? time : new SystemTime(); } - @Override - public String name() { - return inner.name(); - } - @Override public void init(ProcessorContext context, StateStore root) { final String name = name(); this.metrics = context.metrics(); - this.putTime = this.metrics.addLatencySensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); - this.fetchTime = this.metrics.addLatencySensor(metricScope, name, "fetch", Sensor.RecordingLevel.DEBUG); - this.flushTime = this.metrics.addLatencySensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); - this.getTime = this.metrics.addLatencySensor(metricScope, name, "get", Sensor.RecordingLevel.DEBUG); - this.removeTime = this.metrics.addLatencySensor(metricScope, name, "remove", Sensor.RecordingLevel.DEBUG); + this.putTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); + this.fetchTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "fetch", Sensor.RecordingLevel.DEBUG); + this.flushTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); + this.getTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "get", Sensor.RecordingLevel.DEBUG); + this.removeTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "remove", Sensor.RecordingLevel.DEBUG); - final Sensor restoreTime = this.metrics.addLatencySensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); + final Sensor restoreTime = this.metrics.addLatencyAndThroughputSensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); // register and possibly restore the state from the logs final long startNs = time.nanoseconds(); try { @@ -71,16 +69,6 @@ public void init(ProcessorContext context, StateStore root) { } } - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - @Override public byte[] get(final Bytes key) { final long startNs = time.nanoseconds(); @@ -116,11 +104,6 @@ public void put(final Bytes key, final byte[] value) { } } - @Override - public void close() { - inner.close(); - } - @Override public void flush() { final long startNs = time.nanoseconds(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java deleted file mode 100644 index c725c1ac6cefd..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredWindowStore.java +++ /dev/null @@ -1,180 +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.streams.state.internals; - -import org.apache.kafka.common.metrics.Sensor; -import org.apache.kafka.common.utils.Time; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.streams.processor.internals.StreamsMetricsImpl; -import org.apache.kafka.streams.state.WindowStore; -import org.apache.kafka.streams.state.WindowStoreIterator; - -public class MeteredWindowStore implements WindowStore { - - protected final WindowStore inner; - protected final String metricScope; - protected final Time time; - - private Sensor putTime; - private Sensor fetchTime; - private Sensor flushTime; - private Sensor restoreTime; - private StreamsMetricsImpl metrics; - - private ProcessorContext context; - private StateStore root; - private Runnable initDelegate = new Runnable() { - @Override - public void run() { - inner.init(context, root); - } - }; - - private K key; - private V value; - private long timestamp; - private Runnable putDelegate = new Runnable() { - @Override - public void run() { - inner.put(key, value); - } - }; - private Runnable putTsDelegate = new Runnable() { - @Override - public void run() { - inner.put(key, value, timestamp); - } - }; - private Runnable flushDelegate = new Runnable() { - @Override - public void run() { - inner.flush(); - } - }; - - // always wrap the store with the metered store - public MeteredWindowStore(final WindowStore inner, String metricScope, Time time) { - this.inner = inner; - this.metricScope = metricScope; - this.time = time != null ? time : Time.SYSTEM; - } - - @Override - public String name() { - return inner.name(); - } - - @Override - public void init(ProcessorContext context, StateStore root) { - final String name = name(); - this.context = context; - this.root = root; - this.metrics = (StreamsMetricsImpl) context.metrics(); - this.putTime = this.metrics.addLatencySensor(metricScope, name, "put", Sensor.RecordingLevel.DEBUG); - this.fetchTime = this.metrics.addLatencySensor(metricScope, name, "fetch", Sensor.RecordingLevel.DEBUG); - this.flushTime = this.metrics.addLatencySensor(metricScope, name, "flush", Sensor.RecordingLevel.DEBUG); - this.restoreTime = this.metrics.addLatencySensor(metricScope, name, "restore", Sensor.RecordingLevel.DEBUG); - - // register and possibly restore the state from the logs - metrics.measureLatencyNs(time, initDelegate, this.restoreTime); - } - - @Override - public boolean persistent() { - return inner.persistent(); - } - - @Override - public boolean isOpen() { - return inner.isOpen(); - } - - @Override - public WindowStoreIterator fetch(K key, long timeFrom, long timeTo) { - return new MeteredWindowStoreIterator<>(this.inner.fetch(key, timeFrom, timeTo), this.fetchTime); - } - - @Override - public void put(K key, V value) { - this.key = key; - this.value = value; - metrics.measureLatencyNs(time, putDelegate, this.putTime); - } - - @Override - public void put(K key, V value, long timestamp) { - this.key = key; - this.value = value; - this.timestamp = timestamp; - metrics.measureLatencyNs(time, putTsDelegate, this.putTime); - } - - @Override - public void close() { - inner.close(); - } - - @Override - public void flush() { - metrics.measureLatencyNs(time, flushDelegate, this.flushTime); - } - - private class MeteredWindowStoreIterator implements WindowStoreIterator { - - private final WindowStoreIterator iter; - private final Sensor sensor; - private final long startNs; - - public MeteredWindowStoreIterator(WindowStoreIterator iter, Sensor sensor) { - this.iter = iter; - this.sensor = sensor; - this.startNs = time.nanoseconds(); - } - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public KeyValue next() { - return iter.next(); - } - - @Override - public void remove() { - iter.remove(); - } - - @Override - public void close() { - try { - iter.close(); - } finally { - metrics.recordLatency(this.sensor, this.startNs, time.nanoseconds()); - } - } - - } - - WindowStore inner() { - return inner; - } -} 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 6dc08f2247e8b..229ccec056649 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 @@ -106,8 +106,10 @@ synchronized void flush() { private void flush(final LRUNode evicted) { numFlushes++; - log.debug("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}", - name, hits(), misses(), overwrites(), flushes()); + if (log.isTraceEnabled()) { + log.trace("Named cache {} stats on flush: #hits={}, #misses={}, #overwrites={}, #flushes={}", + name, hits(), misses(), overwrites(), flushes()); + } if (listener == null) { throw new IllegalArgumentException("No listener for namespace " + name + " registered with cache"); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index 164b3525d3701..b72bbede4c52b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -35,28 +35,54 @@ public class RocksDBKeyValueStoreSupplier extends AbstractStoreSupplier { - private final boolean enableCaching; + private static final String METRICS_SCOPE = "rocksdb-state"; + private final boolean cached; - public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean enableCaching) { - this(name, keySerde, valueSerde, null, logged, logConfig, enableCaching); + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean cached) { + this(name, keySerde, valueSerde, null, logged, logConfig, cached); } - public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time, boolean logged, Map logConfig, boolean enableCaching) { + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time, boolean logged, Map logConfig, boolean cached) { super(name, keySerde, valueSerde, time, logged, logConfig); - this.enableCaching = enableCaching; + this.cached = cached; } public KeyValueStore get() { - if (!enableCaching) { - RocksDBStore store = new RocksDBStore<>(name, keySerde, valueSerde); - return new MeteredKeyValueStore<>(logged ? store.enableLogging() : store, "rocksdb-state", time); + if (!cached && !logged) { + return new MeteredKeyValueStore<>( + new RocksDBStore<>(name, keySerde, valueSerde), METRICS_SCOPE, time); + } + + // when cached, logged, or both we use a bytes store as the inner most store + final RocksDBStore rocks = new RocksDBStore<>(name, + Serdes.Bytes(), + Serdes.ByteArray()); + + if (cached && logged) { + return new CachingKeyValueStore<>( + new MeteredKeyValueStore<>( + new ChangeLoggingKeyValueBytesStore(rocks), + METRICS_SCOPE, + time), + keySerde, + valueSerde); + } + + if (cached) { + return new CachingKeyValueStore<>( + new MeteredKeyValueStore<>(rocks, METRICS_SCOPE, time), + keySerde, + valueSerde); + + } else { + // logged + return new MeteredKeyValueStore<>( + new ChangeLoggingKeyValueStore<>(rocks, keySerde, valueSerde), + METRICS_SCOPE, + time); } - final RocksDBStore store = new RocksDBStore<>(name, Serdes.Bytes(), Serdes.ByteArray()); - return new CachingKeyValueStore<>(new MeteredKeyValueStore<>(logged ? store.enableLogging() : store, - "rocksdb-state", - time), - keySerde, - valueSerde); } + + } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java index 31956ba486984..ef6ea3c691bd0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStore.java @@ -20,6 +20,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import java.util.List; @@ -95,6 +96,8 @@ public String name() { public void init(ProcessorContext context, StateStore root) { this.context = context; + keySchema.init(ProcessorStateManager.storeChangelogTopic(context.applicationId(), root.name())); + segments.openExisting(context); // register and possibly restore the state from the logs diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java index a8ddc7376f172..22f4a9d44bf3d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStore.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.kstream.internals.SessionKeySerde; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.StateSerdes; @@ -34,6 +35,7 @@ class RocksDBSessionStore implements SessionStore { private final Serde aggSerde; private final SegmentedBytesStore bytesStore; private StateSerdes serdes; + protected String topic; RocksDBSessionStore(final SegmentedBytesStore bytesStore, @@ -55,12 +57,12 @@ public KeyValueIterator, AGG> findSessions(final K key, final long e @Override public void remove(final Windowed key) { - bytesStore.remove(SessionKeySerde.toBinary(key, serdes.keySerializer())); + bytesStore.remove(SessionKeySerde.toBinary(key, serdes.keySerializer(), topic)); } @Override public void put(final Windowed sessionKey, final AGG aggregate) { - bytesStore.put(SessionKeySerde.toBinary(sessionKey, serdes.keySerializer()), aggSerde.serializer().serialize(bytesStore.name(), aggregate)); + bytesStore.put(SessionKeySerde.toBinary(sessionKey, serdes.keySerializer(), topic), aggSerde.serializer().serialize(bytesStore.name(), aggregate)); } @Override @@ -71,7 +73,9 @@ public String name() { @Override @SuppressWarnings("unchecked") public void init(final ProcessorContext context, final StateStore root) { - this.serdes = new StateSerdes<>(bytesStore.name(), + final String storeName = bytesStore.name(); + topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); + this.serdes = new StateSerdes<>(topic, keySerde == null ? (Serde) context.keySerde() : keySerde, aggSerde == null ? (Serde) context.valueSerde() : aggSerde); @@ -121,7 +125,7 @@ public void close() { @Override public Windowed peekNextKey() { final Bytes bytes = bytesIterator.peekNextKey(); - return SessionKeySerde.from(bytes.get(), serdes.keyDeserializer()); + return SessionKeySerde.from(bytes.get(), serdes.keyDeserializer(), serdes.topic()); } @Override @@ -132,7 +136,7 @@ public boolean hasNext() { @Override public KeyValue, AGG> next() { final KeyValue next = bytesIterator.next(); - return KeyValue.pair(SessionKeySerde.from(next.key.get(), serdes.keyDeserializer()), serdes.valueFrom(next.value)); + return KeyValue.pair(SessionKeySerde.from(next.key.get(), serdes.keyDeserializer(), serdes.topic()), serdes.valueFrom(next.value)); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java index 76454724ba7fd..10ebf65b217e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplier.java @@ -5,9 +5,9 @@ * 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 - * + *

      + * 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. @@ -18,6 +18,8 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.SessionStore; @@ -35,13 +37,14 @@ public class RocksDBSessionStoreSupplier extends AbstractStoreSupplier implements WindowStoreSupplier { private static final int NUM_SEGMENTS = 3; + public static final String METRIC_SCOPE = "rocksdb-session-store"; private final long retentionPeriod; - private final boolean enableCaching; + private final boolean cached; - public RocksDBSessionStoreSupplier(String name, long retentionPeriod, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean enableCaching) { + public RocksDBSessionStoreSupplier(String name, long retentionPeriod, Serde keySerde, Serde valueSerde, boolean logged, Map logConfig, boolean cached) { super(name, keySerde, valueSerde, Time.SYSTEM, logged, logConfig); this.retentionPeriod = retentionPeriod; - this.enableCaching = enableCaching; + this.cached = cached; } public String name() { @@ -49,16 +52,41 @@ public String name() { } public SessionStore get() { - final RocksDBSegmentedBytesStore bytesStore = new RocksDBSegmentedBytesStore(name, + final SessionKeySchema keySchema = new SessionKeySchema(); + final RocksDBSegmentedBytesStore segmented = new RocksDBSegmentedBytesStore(name, retentionPeriod, NUM_SEGMENTS, - new SessionKeySchema()); - final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore, "rocksdb-session-store", time); - if (enableCaching) { - return new CachingSessionStore<>(metered, keySerde, valueSerde); + keySchema + ); + + if (cached && logged) { + final ChangeLoggingSegmentedBytesStore logged = new ChangeLoggingSegmentedBytesStore(segmented); + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged, + METRIC_SCOPE, time); + final RocksDBSessionStore sessionStore + = new RocksDBSessionStore<>(metered, Serdes.Bytes(), Serdes.ByteArray()); + + return new CachingSessionStore<>(sessionStore, keySerde, valueSerde); + } + + if (cached) { + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(segmented, + METRIC_SCOPE, time); + final RocksDBSessionStore sessionStore + = new RocksDBSessionStore<>(metered, Serdes.Bytes(), Serdes.ByteArray()); + + return new CachingSessionStore<>(sessionStore, keySerde, valueSerde); } - return new RocksDBSessionStore<>(metered, keySerde, valueSerde); + + if (logged) { + final ChangeLoggingSegmentedBytesStore logged = new ChangeLoggingSegmentedBytesStore(segmented); + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(logged, + METRIC_SCOPE, time); + return new RocksDBSessionStore<>(metered, keySerde, valueSerde); + } + + return new RocksDBSessionStore<>( + new MeteredSegmentedBytesStore(segmented, METRIC_SCOPE, time), keySerde, valueSerde); } 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 8b838d02dc2f2..a778cd8cea7cd 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 @@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.RocksDBConfigSetter; @@ -35,6 +36,7 @@ import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; import org.rocksdb.FlushOptions; +import org.rocksdb.InfoLogLevel; import org.rocksdb.Options; import org.rocksdb.RocksDB; import org.rocksdb.RocksDBException; @@ -92,31 +94,23 @@ public class RocksDBStore implements KeyValueStore { private WriteOptions wOptions; private FlushOptions fOptions; - private boolean loggingEnabled = false; - - private StoreChangeLogger changeLogger; - protected volatile boolean open = false; - public KeyValueStore enableLogging() { - loggingEnabled = true; - - return this; - } - - public RocksDBStore(String name, Serde keySerde, Serde valueSerde) { + RocksDBStore(String name, Serde keySerde, Serde valueSerde) { this(name, DB_FILE_DIR, keySerde, valueSerde); } - - public RocksDBStore(String name, String parentDir, Serde keySerde, Serde valueSerde) { + RocksDBStore(String name, String parentDir, Serde keySerde, Serde valueSerde) { this.name = name; this.parentDir = parentDir; this.keySerde = keySerde; this.valueSerde = valueSerde; + } + @SuppressWarnings("unchecked") + public void openDB(ProcessorContext context) { // initialize the default rocksdb options - BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); + final BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); tableConfig.setBlockSize(BLOCK_SIZE); @@ -128,17 +122,22 @@ public RocksDBStore(String name, String parentDir, Serde keySerde, Serde v options.setMaxWriteBufferNumber(MAX_WRITE_BUFFERS); options.setCreateIfMissing(true); options.setErrorIfExists(false); - + options.setInfoLogLevel(InfoLogLevel.ERROR_LEVEL); + // this is the recommended way to increase parallelism in RocksDb + // note that the current implementation of setIncreaseParallelism affects the number + // of compaction threads but not flush threads (the latter remains one). Also + // the parallelism value needs to be at least two because of the code in + // https://github.com/facebook/rocksdb/blob/62ad0a9b19f0be4cefa70b6b32876e764b7f3c11/util/options.cc#L580 + // subtracts one from the value passed to determine the number of compaction threads + // (this could be a bug in the RocksDB code and their devs have been contacted). + options.setIncreaseParallelism(Math.max(Runtime.getRuntime().availableProcessors(), 2)); wOptions = new WriteOptions(); wOptions.setDisableWAL(true); fOptions = new FlushOptions(); fOptions.setWaitForFlush(true); - } - @SuppressWarnings("unchecked") - public void openDB(ProcessorContext context) { final Map configs = context.appConfigs(); final Class configSetterClass = (Class) configs.get(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG); if (configSetterClass != null) { @@ -147,9 +146,10 @@ public void openDB(ProcessorContext context) { } // we need to construct the serde while opening DB since // it is also triggered by windowed DB segments without initialization - this.serdes = new StateSerdes<>(name, - keySerde == null ? (Serde) context.keySerde() : keySerde, - valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + this.serdes = new StateSerdes<>( + ProcessorStateManager.storeChangelogTopic(context.applicationId(), name), + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); this.dbDir = new File(new File(context.stateDir(), parentDir), this.name); this.db = openDB(this.dbDir, this.options, TTL_SECONDS); @@ -159,10 +159,9 @@ public void init(ProcessorContext context, StateStore root) { // open the DB dir openDB(context); - this.changeLogger = this.loggingEnabled ? new StoreChangeLogger<>(name, context, WindowStoreUtils.INNER_SERDES) : null; // value getter should always read directly from rocksDB // since it is only for values that are already flushed - context.register(root, loggingEnabled, new StateRestoreCallback() { + context.register(root, false, new StateRestoreCallback() { @Override public void restore(byte[] key, byte[] value) { @@ -235,10 +234,6 @@ public synchronized void put(K key, V value) { byte[] rawKey = serdes.rawKey(key); byte[] rawValue = serdes.rawValue(value); putInternal(rawKey, rawValue); - - if (loggingEnabled) { - changeLogger.logChange(Bytes.wrap(rawKey), rawValue); - } } @Override @@ -278,9 +273,6 @@ public void putAll(List> entries) { } else { final byte[] value = serdes.rawValue(entry.value); batch.put(rawKey, value); - if (loggingEnabled) { - changeLogger.logChange(Bytes.wrap(rawKey), value); - } } } db.write(wOptions, batch); @@ -372,6 +364,7 @@ public synchronized void close() { if (!open) { return; } + open = false; closeOpenIterators(); options.close(); @@ -476,5 +469,4 @@ public synchronized boolean hasNext() { return super.hasNext() && comparator.compare(super.peekRawKey(), this.rawToKey) <= 0; } } - } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index a2a420e1c3363..ffd3061274026 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -20,10 +20,12 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; @@ -33,7 +35,6 @@ class RocksDBWindowStore implements WindowStore { - private final String name; private final SegmentedBytesStore bytesStore; private final boolean retainDuplicates; private final Serde keySerde; @@ -43,8 +44,15 @@ class RocksDBWindowStore implements WindowStore { private StateSerdes serdes; - RocksDBWindowStore(String name, boolean retainDuplicates, Serde keySerde, Serde valueSerde, final SegmentedBytesStore bytesStore) { - this.name = name; + static RocksDBWindowStore bytesStore(final SegmentedBytesStore inner, final boolean retainDuplicates) { + return new RocksDBWindowStore<>(inner, Serdes.Bytes(), Serdes.ByteArray(), retainDuplicates); + } + + + RocksDBWindowStore(final SegmentedBytesStore bytesStore, + final Serde keySerde, + final Serde valueSerde, + final boolean retainDuplicates) { this.keySerde = keySerde; this.valueSerde = valueSerde; this.retainDuplicates = retainDuplicates; @@ -54,7 +62,7 @@ class RocksDBWindowStore implements WindowStore { @Override public String name() { - return name; + return bytesStore.name(); } @Override @@ -62,7 +70,7 @@ public String name() { public void init(final ProcessorContext context, final StateStore root) { this.context = context; // construct the serde - this.serdes = new StateSerdes<>(name, + this.serdes = new StateSerdes<>(ProcessorStateManager.storeChangelogTopic(context.applicationId(), bytesStore.name()), keySerde == null ? (Serde) context.keySerde() : keySerde, valueSerde == null ? (Serde) context.valueSerde() : valueSerde); @@ -147,6 +155,14 @@ public void remove() { public void close() { actual.close(); } + + @Override + public Long peekNextKey() { + if (!actual.hasNext()) { + throw new NoSuchElementException(); + } + return WindowStoreUtils.timestampFromBinaryKey(actual.peekNextKey().get()); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 84f1734fb7c8e..abaaffd2bff84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.state.WindowStore; @@ -58,23 +59,35 @@ public String name() { } public WindowStore get() { - final RocksDBSegmentedBytesStore bytesStore = new RocksDBSegmentedBytesStore(name, retentionPeriod, numSegments, new WindowStoreKeySchema()); - if (!enableCaching) { - final RocksDBWindowStore segmentedStore = new RocksDBWindowStore<>(name, retainDuplicates, keySerde, valueSerde, - logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore); - return new MeteredWindowStore<>(segmentedStore, "rocksdb-window", time); - } + return maybeWrapCaching( + maybeWrapLogged( + new RocksDBSegmentedBytesStore( + name, + retentionPeriod, + numSegments, + new WindowStoreKeySchema() + ))); - return new CachingWindowStore<>(new MeteredSegmentedBytesStore(logged ? new ChangeLoggingSegmentedBytesStore(bytesStore) - : bytesStore, - "rocksdb-window", - time), - keySerde, valueSerde, windowSize); } @Override public long retentionPeriod() { return retentionPeriod; } + + private SegmentedBytesStore maybeWrapLogged(final SegmentedBytesStore inner) { + if (!logged) { + return inner; + } + return new ChangeLoggingSegmentedBytesStore(inner); + } + + private WindowStore maybeWrapCaching(final SegmentedBytesStore inner) { + final MeteredSegmentedBytesStore metered = new MeteredSegmentedBytesStore(inner, "rocksdb-window", time); + if (!enableCaching) { + return new RocksDBWindowStore<>(metered, keySerde, valueSerde, retainDuplicates); + } + final RocksDBWindowStore windowed = RocksDBWindowStore.bytesStore(metered, retainDuplicates); + return new CachingWindowStore<>(windowed, keySerde, valueSerde, windowSize); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java index ab1099ea85e1f..b2957dae1e46b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SegmentedBytesStore.java @@ -67,6 +67,14 @@ public interface SegmentedBytesStore extends StateStore { interface KeySchema { + + /** + * Initialized the schema with a topic. + * + * @param topic a topic name + */ + void init(final String topic); + /** * Given a record-key and a time, construct a Segmented key that represents * the upper range of keys to search when performing range queries. diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java index 13cdf6c16b523..72eb802f1326e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/Segments.java @@ -40,7 +40,8 @@ class Segments { private final int numSegments; private final long segmentInterval; private final SimpleDateFormat formatter; - private long currentSegmentId = -1L; + private long minSegmentId = Long.MAX_VALUE; + private long maxSegmentId = -1L; Segments(final String name, final long retentionPeriod, final int numSegments) { this.name = name; @@ -64,7 +65,7 @@ Segment getSegmentForTimestamp(final long timestamp) { } Segment getOrCreateSegment(final long segmentId, final ProcessorContext context) { - if (segmentId > currentSegmentId || segmentId > currentSegmentId - numSegments) { + if (segmentId > maxSegmentId - numSegments) { final long key = segmentId % numSegments; final Segment segment = segments.get(key); if (!isSegment(segment, segmentId)) { @@ -74,7 +75,10 @@ Segment getOrCreateSegment(final long segmentId, final ProcessorContext context) Segment newSegment = new Segment(segmentName(segmentId), name, segmentId); newSegment.openDB(context); segments.put(key, newSegment); - currentSegmentId = segmentId > currentSegmentId ? segmentId : currentSegmentId; + maxSegmentId = segmentId > maxSegmentId ? segmentId : maxSegmentId; + if (minSegmentId == Long.MAX_VALUE) { + minSegmentId = maxSegmentId; + } } return segments.get(key); } else { @@ -112,8 +116,8 @@ void openExisting(final ProcessorContext context) { } List segments(final long timeFrom, final long timeTo) { - final long segFrom = segmentId(Math.max(0L, timeFrom)); - final long segTo = segmentId(Math.min(currentSegmentId * segmentInterval, Math.max(0, timeTo))); + final long segFrom = Math.max(minSegmentId, segmentId(Math.max(0L, timeFrom))); + final long segTo = Math.min(maxSegmentId, segmentId(Math.min(maxSegmentId * segmentInterval, Math.max(0, timeTo)))); final List segments = new ArrayList<>(); for (long segmentId = segFrom; segmentId <= segTo; segmentId++) { @@ -154,9 +158,9 @@ private boolean isSegment(final Segment store, long segmentId) { } private void cleanup(final long segmentId) { - final long oldestSegmentId = currentSegmentId < segmentId + final long oldestSegmentId = maxSegmentId < segmentId ? segmentId - numSegments - : currentSegmentId - numSegments; + : maxSegmentId - numSegments; for (Map.Entry segmentEntry : segments.entrySet()) { final Segment segment = segmentEntry.getValue(); @@ -166,6 +170,9 @@ private void cleanup(final long segmentId) { segment.destroy(); } } + if (oldestSegmentId > minSegmentId) { + minSegmentId = oldestSegmentId + 1; + } } private long segmentIdFromSegmentName(String segmentName) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.java new file mode 100644 index 0000000000000..d76e8a4568e7b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIterator.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 org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; + +import java.util.NoSuchElementException; + +class SerializedKeyValueIterator implements KeyValueIterator { + + private final KeyValueIterator bytesIterator; + private final StateSerdes serdes; + + SerializedKeyValueIterator(final KeyValueIterator bytesIterator, + final StateSerdes serdes) { + + this.bytesIterator = bytesIterator; + this.serdes = serdes; + } + + @Override + public void close() { + bytesIterator.close(); + } + + @Override + public K peekNextKey() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final Bytes bytes = bytesIterator.peekNextKey(); + return serdes.keyFrom(bytes.get()); + } + + @Override + public boolean hasNext() { + return bytesIterator.hasNext(); + } + + @Override + public KeyValue next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + final KeyValue next = bytesIterator.next(); + return KeyValue.pair(serdes.keyFrom(next.key.get()), serdes.valueFrom(next.value)); + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove not supported by SerializedKeyValueIterator"); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java index 604abb321382e..698970a54054b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/SessionKeySchema.java @@ -27,17 +27,23 @@ class SessionKeySchema implements SegmentedBytesStore.KeySchema { + private String topic; + + @Override + public void init(final String topic) { + this.topic = topic; + } @Override public Bytes upperRange(final Bytes key, final long to) { final Windowed sessionKey = new Windowed<>(key, new SessionWindow(to, Long.MAX_VALUE)); - return SessionKeySerde.toBinary(sessionKey, Serdes.Bytes().serializer()); + return SessionKeySerde.toBinary(sessionKey, Serdes.Bytes().serializer(), topic); } @Override public Bytes lowerRange(final Bytes key, final long from) { final Windowed sessionKey = new Windowed<>(key, new SessionWindow(0, Math.max(0, from))); - return SessionKeySerde.toBinary(sessionKey, Serdes.Bytes().serializer()); + return SessionKeySerde.toBinary(sessionKey, Serdes.Bytes().serializer(), topic); } @Override @@ -50,15 +56,15 @@ public HasNextCondition hasNextCondition(final Bytes binaryKey, final long from, return new HasNextCondition() { @Override public boolean hasNext(final KeyValueIterator iterator) { - if (iterator.hasNext()) { + while (iterator.hasNext()) { final Bytes bytes = iterator.peekNextKey(); - final Bytes keyBytes = Bytes.wrap(SessionKeySerde.extractKeyBytes(bytes.get())); - if (!keyBytes.equals(binaryKey)) { - return false; + final Windowed windowedKey = SessionKeySerde.fromBytes(bytes); + if (windowedKey.key().equals(binaryKey) + && windowedKey.window().end() >= from + && windowedKey.window().start() <= to) { + return true; } - final long start = SessionKeySerde.extractStart(bytes.get()); - final long end = SessionKeySerde.extractEnd(bytes.get()); - return end >= from && start <= to; + iterator.next(); } return false; } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index 44cda5be3887b..39c8270600bd6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; @@ -58,7 +57,7 @@ void logChange(final K key, final V value) { if (collector != null) { final Serializer keySerializer = serialization.keySerializer(); final Serializer valueSerializer = serialization.valueSerializer(); - collector.send(new ProducerRecord<>(this.topic, this.partition, context.timestamp(), key, value), keySerializer, valueSerializer); + collector.send(this.topic, key, value, this.partition, context.timestamp(), keySerializer, valueSerializer); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreKeySchema.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreKeySchema.java index 093161ea6ca64..9d32592c5593d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreKeySchema.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreKeySchema.java @@ -24,13 +24,12 @@ import java.util.List; class WindowStoreKeySchema implements RocksDBSegmentedBytesStore.KeySchema { - private static final HasNextCondition ITERATOR_HAS_NEXT = new HasNextCondition() { - @Override - public boolean hasNext(final KeyValueIterator iterator) { - return iterator.hasNext(); - } - }; - private final StateSerdes serdes = new StateSerdes<>("window-store-key-schema", Serdes.Bytes(), Serdes.ByteArray()); + private StateSerdes serdes; + + @Override + public void init(final String topic) { + serdes = new StateSerdes<>(topic, Serdes.Bytes(), Serdes.ByteArray()); + } @Override public Bytes upperRange(final Bytes key, final long to) { @@ -49,7 +48,24 @@ public long segmentTimestamp(final Bytes key) { @Override public HasNextCondition hasNextCondition(final Bytes binaryKey, final long from, final long to) { - return ITERATOR_HAS_NEXT; + return new HasNextCondition() { + @Override + public boolean hasNext(final KeyValueIterator iterator) { + while (iterator.hasNext()) { + final Bytes bytes = iterator.peekNextKey(); + final Bytes keyBytes = WindowStoreUtils.bytesKeyFromBinaryKey(bytes.get()); + final long time = WindowStoreUtils.timestampFromBinaryKey(bytes.get()); + if (keyBytes.equals(binaryKey) + && time >= from + && time <= to) { + return true; + } + iterator.next(); + } + + return false; + } + }; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java index 1ea6bef7df4aa..0f491ac7e8bbb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java @@ -34,12 +34,18 @@ public class WindowStoreUtils { /** Inner byte array serde used for segments */ public static final Serde INNER_KEY_SERDE = Serdes.Bytes(); public static final Serde INNER_VALUE_SERDE = Serdes.ByteArray(); - public static final StateSerdes INNER_SERDES = new StateSerdes<>("rocksDB-inner", INNER_KEY_SERDE, INNER_VALUE_SERDE); + static StateSerdes getInnerStateSerde(final String topic) { + return new StateSerdes<>(topic, INNER_KEY_SERDE, INNER_VALUE_SERDE); + } + public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes serdes) { byte[] serializedKey = serdes.rawKey(key); + return toBinaryKey(serializedKey, timestamp, seqnum); + } + static byte[] toBinaryKey(byte[] serializedKey, final long timestamp, final int seqnum) { ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE); buf.put(serializedKey); buf.putLong(timestamp); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java new file mode 100644 index 0000000000000..3d80b98d69744 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WrappedStateStore.java @@ -0,0 +1,90 @@ +/** + * 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 org.apache.kafka.streams.errors.InvalidStateStoreException; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; + +/** + * A storage engine wrapper for utilities like logging, caching, and metering. + */ +interface WrappedStateStore extends StateStore { + + /** + * Return the inner storage engine + * + * @return wrapped inner storage engine + */ + StateStore inner(); + + abstract class AbstractWrappedStateStore implements WrappedStateStore { + final StateStore innerState; + + AbstractWrappedStateStore(StateStore inner) { + this.innerState = inner; + } + + @Override + public void init(ProcessorContext context, StateStore root) { + innerState.init(context, root); + } + + @Override + public String name() { + return innerState.name(); + } + + @Override + public boolean persistent() { + return innerState.persistent(); + } + + @Override + public boolean isOpen() { + return innerState.isOpen(); + } + + void validateStoreOpen() { + if (!innerState.isOpen()) { + throw new InvalidStateStoreException("Store " + innerState.name() + " is currently closed."); + } + } + + @Override + public StateStore inner() { + if (innerState instanceof WrappedStateStore) { + return ((WrappedStateStore) innerState).inner(); + } + return innerState; + } + + @Override + public void flush() { + innerState.flush(); + } + + @Override + public void close() { + innerState.close(); + } + } + + +} 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 41277c7f264b4..5dae8ddf580ad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.Metric; import org.apache.kafka.common.MetricName; import org.apache.kafka.common.config.ConfigException; @@ -201,6 +202,7 @@ public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() throws Excepti final Properties props = new Properties(); props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "appId"); props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + props.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); final KStreamBuilder builder = new KStreamBuilder(); final CountDownLatch latch = new CountDownLatch(1); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java index 85b851d12964f..a844a3cba259c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/GlobalKTableIntegrationTest.java @@ -81,7 +81,7 @@ public String apply(final Long value1, final String value2) { private ForeachAction foreachAction; @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -90,10 +90,10 @@ public void before() { streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); streamsConfiguration .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); globalTable = builder.globalTable(Serdes.Long(), Serdes.String(), globalOne, globalStore); stream = builder.stream(Serdes.String(), Serdes.Long(), inputStream); table = builder.table(Serdes.String(), Serdes.Long(), inputTable, "table"); @@ -212,7 +212,7 @@ public boolean conditionMet() { } - private void createTopics() { + private void createTopics() throws InterruptedException { inputStream = "input-stream-" + testNo; inputTable = "input-table-" + testNo; globalOne = "globalOne-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java index 52bd6e7e00729..20556c4073622 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -87,6 +87,8 @@ public void before() { streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); } private Properties getTopicConfigProperties(final String changelog) { @@ -219,4 +221,4 @@ public Iterable apply(String value) { final long retention = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS) + durationMs; assertEquals(retention, Long.parseLong(properties.getProperty(LogConfig.RetentionMsProp()))); } -} \ No newline at end of file +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index b443abc08ede8..0c052f570d13c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -121,12 +121,12 @@ public static void setupConfigsAndUtils() throws Exception { RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - STREAMS_CONFIG.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); STREAMS_CONFIG.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); STREAMS_CONFIG.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); zkUtils = ZkUtils.apply(CLUSTER.zKConnectString(), 30000, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java index 9397e0331f474..f2a767ceb514e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationDedupIntegrationTest.java @@ -72,7 +72,7 @@ public class KStreamAggregationDedupIntegrationTest { @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -267,7 +267,7 @@ private void produceMessages(long timestamp) } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; outputTopic = "output-" + testNo; CLUSTER.createTopic(streamOneInput, 3, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java index 0833f3c8d1fe4..f6708a7f02d35 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamAggregationIntegrationTest.java @@ -89,7 +89,7 @@ public class KStreamAggregationIntegrationTest { private KStream stream; @Before - public void before() { + public void before() throws InterruptedException { testNo++; builder = new KStreamBuilder(); createTopics(); @@ -101,7 +101,7 @@ public void before() { streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); - + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); final KeyValueMapper mapper = MockKeyValueMapper.SelectValueMapper(); stream = builder.stream(Serdes.Integer(), Serdes.String(), streamOneInput); @@ -637,7 +637,7 @@ private void produceMessages(final long timestamp) } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; outputTopic = "output-" + testNo; userSessionsStream = userSessionsStream + "-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java index 3618f1542c271..0a16494397c71 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamKTableJoinIntegrationTest.java @@ -74,7 +74,7 @@ public class KStreamKTableJoinIntegrationTest { private Properties streamsConfiguration; @Before - public void before() { + public void before() throws InterruptedException { testNo++; userClicksTopic = "user-clicks-" + testNo; userRegionsTopic = "user-regions-" + testNo; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java index e8a042ad7c670..43e5d879f311d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java @@ -93,7 +93,7 @@ public static Object[] data() { } @Before - public void before() { + public void before() throws InterruptedException { testNo++; String applicationId = "kstream-repartition-join-test-" + testNo; builder = new KStreamBuilder(); @@ -146,7 +146,7 @@ public void shouldCorrectlyRepartitionOnJoinOperations() throws Exception { verifyLeftJoin(leftJoin); } - private ExpectedOutputOnTopic mapStreamOneAndJoin() { + private ExpectedOutputOnTopic mapStreamOneAndJoin() throws InterruptedException { String mapOneStreamAndJoinOutput = "map-one-join-output-" + testNo; doJoin(streamOne.map(keyMapper), streamTwo, mapOneStreamAndJoinOutput); return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, mapOneStreamAndJoinOutput); @@ -160,7 +160,6 @@ private ExpectedOutputOnTopic mapBothStreamsAndJoin() throws Exception { return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, "map-both-streams-and-join-" + testNo); } - private ExpectedOutputOnTopic mapMapJoin() throws Exception { final KStream mapMapStream = streamOne.map( new KeyValueMapper>() { @@ -178,8 +177,7 @@ public KeyValue apply(final Long key, final Integer value) { return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); } - - public ExpectedOutputOnTopic selectKeyAndJoin() throws ExecutionException, InterruptedException { + private ExpectedOutputOnTopic selectKeyAndJoin() throws ExecutionException, InterruptedException { final KStream keySelected = streamOne.selectKey(MockKeyValueMapper.SelectValueMapper()); @@ -189,7 +187,6 @@ public ExpectedOutputOnTopic selectKeyAndJoin() throws ExecutionException, Inter return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); } - private ExpectedOutputOnTopic flatMapJoin() throws Exception { final KStream flatMapped = streamOne.flatMap( new KeyValueMapper>>() { @@ -221,7 +218,7 @@ private ExpectedOutputOnTopic joinMappedRhsStream() throws Exception { return new ExpectedOutputOnTopic(Arrays.asList("A:1", "B:2", "C:3", "D:4", "E:5"), output); } - public ExpectedOutputOnTopic mapBothStreamsAndLeftJoin() throws Exception { + private ExpectedOutputOnTopic mapBothStreamsAndLeftJoin() throws Exception { final KStream map1 = streamOne.map(keyMapper); final KStream map2 = streamTwo.map(MockKeyValueMapper.NoOpKeyValueMapper()); @@ -312,7 +309,6 @@ private void produceMessages() } - private void produceStreamTwoInputTo(final String streamTwoInput) throws ExecutionException, InterruptedException { IntegrationTestUtils.produceKeyValuesSynchronously( @@ -350,13 +346,13 @@ private void produceToStreamOne() mockTime); } - private void createTopics() { + private void createTopics() throws InterruptedException { streamOneInput = "stream-one-" + testNo; streamTwoInput = "stream-two-" + testNo; streamFourInput = "stream-four-" + testNo; - CLUSTER.createTopic(streamOneInput); - CLUSTER.createTopic(streamTwoInput); - CLUSTER.createTopic(streamFourInput); + CLUSTER.createTopic(streamOneInput, 2, 1); + CLUSTER.createTopic(streamTwoInput, 2, 1); + CLUSTER.createTopic(streamFourInput, 2, 1); } @@ -395,7 +391,7 @@ private void verifyCorrectOutput(final List expectedMessages, private void doJoin(final KStream lhs, final KStream rhs, - final String outputTopic) { + final String outputTopic) throws InterruptedException { CLUSTER.createTopic(outputTopic); lhs.join(rhs, TOSTRING_JOINER, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java index 2171de1e1e075..fe73b73eb2275 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamsFineGrainedAutoResetIntegrationTest.java @@ -19,12 +19,16 @@ import kafka.utils.MockTime; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serde; 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.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; @@ -41,7 +45,9 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; @@ -53,35 +59,71 @@ public class KStreamsFineGrainedAutoResetIntegrationTest { private static final int NUM_BROKERS = 1; private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic"; + private static final String OUTPUT_TOPIC_0 = "outputTopic_0"; + private static final String OUTPUT_TOPIC_1 = "outputTopic_1"; + private static final String OUTPUT_TOPIC_2 = "outputTopic_2"; @ClassRule public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); private final MockTime mockTime = CLUSTER.time; - private static final String TOPIC_1 = "topic-1"; - private static final String TOPIC_2 = "topic-2"; - private static final String TOPIC_A = "topic-A"; - private static final String TOPIC_C = "topic-C"; - private static final String TOPIC_Y = "topic-Y"; - private static final String TOPIC_Z = "topic-Z"; + private static final String TOPIC_1_0 = "topic-1_0"; + private static final String TOPIC_2_0 = "topic-2_0"; + private static final String TOPIC_A_0 = "topic-A_0"; + private static final String TOPIC_C_0 = "topic-C_0"; + private static final String TOPIC_Y_0 = "topic-Y_0"; + private static final String TOPIC_Z_0 = "topic-Z_0"; + private static final String TOPIC_1_1 = "topic-1_1"; + private static final String TOPIC_2_1 = "topic-2_1"; + private static final String TOPIC_A_1 = "topic-A_1"; + private static final String TOPIC_C_1 = "topic-C_1"; + private static final String TOPIC_Y_1 = "topic-Y_1"; + private static final String TOPIC_Z_1 = "topic-Z_1"; + private static final String TOPIC_1_2 = "topic-1_2"; + private static final String TOPIC_2_2 = "topic-2_2"; + private static final String TOPIC_A_2 = "topic-A_2"; + private static final String TOPIC_C_2 = "topic-C_2"; + private static final String TOPIC_Y_2 = "topic-Y_2"; + private static final String TOPIC_Z_2 = "topic-Z_2"; private static final String NOOP = "noop"; private final Serde stringSerde = Serdes.String(); private static final String STRING_SERDE_CLASSNAME = Serdes.String().getClass().getName(); private Properties streamsConfiguration; + private final String topic1TestMessage = "topic-1 test"; + private final String topic2TestMessage = "topic-2 test"; + private final String topicATestMessage = "topic-A test"; + private final String topicCTestMessage = "topic-C test"; + private final String topicYTestMessage = "topic-Y test"; + private final String topicZTestMessage = "topic-Z test"; + @BeforeClass public static void startKafkaCluster() throws Exception { - CLUSTER.createTopic(TOPIC_1); - CLUSTER.createTopic(TOPIC_2); - CLUSTER.createTopic(TOPIC_A); - CLUSTER.createTopic(TOPIC_C); - CLUSTER.createTopic(TOPIC_Y); - CLUSTER.createTopic(TOPIC_Z); + CLUSTER.createTopic(TOPIC_1_0); + CLUSTER.createTopic(TOPIC_2_0); + CLUSTER.createTopic(TOPIC_A_0); + CLUSTER.createTopic(TOPIC_C_0); + CLUSTER.createTopic(TOPIC_Y_0); + CLUSTER.createTopic(TOPIC_Z_0); + CLUSTER.createTopic(TOPIC_1_1); + CLUSTER.createTopic(TOPIC_2_1); + CLUSTER.createTopic(TOPIC_A_1); + CLUSTER.createTopic(TOPIC_C_1); + CLUSTER.createTopic(TOPIC_Y_1); + CLUSTER.createTopic(TOPIC_Z_1); + CLUSTER.createTopic(TOPIC_1_2); + CLUSTER.createTopic(TOPIC_2_2); + CLUSTER.createTopic(TOPIC_A_2); + CLUSTER.createTopic(TOPIC_C_2); + CLUSTER.createTopic(TOPIC_Y_2); + CLUSTER.createTopic(TOPIC_Z_2); CLUSTER.createTopic(NOOP); CLUSTER.createTopic(DEFAULT_OUTPUT_TOPIC); - + CLUSTER.createTopic(OUTPUT_TOPIC_0); + CLUSTER.createTopic(OUTPUT_TOPIC_1); + CLUSTER.createTopic(OUTPUT_TOPIC_2); } @Before @@ -102,41 +144,64 @@ public void setUp() throws Exception { } @Test - public void shouldOnlyReadRecordsWhereEarliestSpecified() throws Exception { + public void shouldOnlyReadRecordsWhereEarliestSpecifiedWithNoCommittedOffsetsWithGlobalAutoOffsetResetLatest() throws Exception { + streamsConfiguration.put(StreamsConfig.consumerPrefix(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG), "latest"); + + final List expectedReceivedValues = Arrays.asList(topic1TestMessage, topic2TestMessage); + shouldOnlyReadForEarliest("_0", TOPIC_1_0, TOPIC_2_0, TOPIC_A_0, TOPIC_C_0, TOPIC_Y_0, TOPIC_Z_0, OUTPUT_TOPIC_0, expectedReceivedValues); + } + + @Test + public void shouldOnlyReadRecordsWhereEarliestSpecifiedWithNoCommittedOffsetsWithDefaultGlobalAutoOffsetResetEarliest() throws Exception { + final List expectedReceivedValues = Arrays.asList(topic1TestMessage, topic2TestMessage, topicYTestMessage, topicZTestMessage); + shouldOnlyReadForEarliest("_1", TOPIC_1_1, TOPIC_2_1, TOPIC_A_1, TOPIC_C_1, TOPIC_Y_1, TOPIC_Z_1, OUTPUT_TOPIC_1, expectedReceivedValues); + } + + @Test + public void shouldOnlyReadRecordsWhereEarliestSpecifiedWithInvalidCommittedOffsets() throws Exception { + commitInvalidOffsets(); + + final List expectedReceivedValues = Arrays.asList(topic1TestMessage, topic2TestMessage, topicYTestMessage, topicZTestMessage); + shouldOnlyReadForEarliest("_2", TOPIC_1_2, TOPIC_2_2, TOPIC_A_2, TOPIC_C_2, TOPIC_Y_2, TOPIC_Z_2, OUTPUT_TOPIC_2, expectedReceivedValues); + } + + private void shouldOnlyReadForEarliest( + final String topicSuffix, + final String topic1, + final String topic2, + final String topicA, + final String topicC, + final String topicY, + final String topicZ, + final String outputTopic, + final List expectedReceivedValues) throws Exception { + final KStreamBuilder builder = new KStreamBuilder(); - final KStream pattern1Stream = builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-\\d")); - final KStream pattern2Stream = builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-[A-D]")); - final KStream namedTopicsStream = builder.stream(TOPIC_Y, TOPIC_Z); + final KStream pattern1Stream = builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-\\d" + topicSuffix)); + final KStream pattern2Stream = builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-[A-D]" + topicSuffix)); + final KStream namedTopicsStream = builder.stream(topicY, topicZ); - pattern1Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC); - pattern2Stream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC); - namedTopicsStream.to(stringSerde, stringSerde, DEFAULT_OUTPUT_TOPIC); + pattern1Stream.to(stringSerde, stringSerde, outputTopic); + pattern2Stream.to(stringSerde, stringSerde, outputTopic); + namedTopicsStream.to(stringSerde, stringSerde, outputTopic); final Properties producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(), StringSerializer.class, StringSerializer.class); - final String topic1TestMessage = "topic-1 test"; - final String topic2TestMessage = "topic-2 test"; - final String topicATestMessage = "topic-A test"; - final String topicCTestMessage = "topic-C test"; - final String topicYTestMessage = "topic-Y test"; - final String topicZTestMessage = "topic-Z test"; - - IntegrationTestUtils.produceValuesSynchronously(TOPIC_1, Collections.singletonList(topic1TestMessage), producerConfig, mockTime); - IntegrationTestUtils.produceValuesSynchronously(TOPIC_2, Collections.singletonList(topic2TestMessage), producerConfig, mockTime); - IntegrationTestUtils.produceValuesSynchronously(TOPIC_A, Collections.singletonList(topicATestMessage), producerConfig, mockTime); - IntegrationTestUtils.produceValuesSynchronously(TOPIC_C, Collections.singletonList(topicCTestMessage), producerConfig, mockTime); - IntegrationTestUtils.produceValuesSynchronously(TOPIC_Y, Collections.singletonList(topicYTestMessage), producerConfig, mockTime); - IntegrationTestUtils.produceValuesSynchronously(TOPIC_Z, Collections.singletonList(topicZTestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topic1, Collections.singletonList(topic1TestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topic2, Collections.singletonList(topic2TestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topicA, Collections.singletonList(topicATestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topicC, Collections.singletonList(topicCTestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topicY, Collections.singletonList(topicYTestMessage), producerConfig, mockTime); + IntegrationTestUtils.produceValuesSynchronously(topicZ, Collections.singletonList(topicZTestMessage), producerConfig, mockTime); final Properties consumerConfig = TestUtils.consumerConfig(CLUSTER.bootstrapServers(), StringDeserializer.class, StringDeserializer.class); final KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration); streams.start(); - final List expectedReceivedValues = Arrays.asList(topic1TestMessage, topic2TestMessage, topicYTestMessage, topicZTestMessage); - final List> receivedKeyValues = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(consumerConfig, DEFAULT_OUTPUT_TOPIC, 4); - final List actualValues = new ArrayList<>(4); + final List> receivedKeyValues = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived(consumerConfig, outputTopic, expectedReceivedValues.size()); + final List actualValues = new ArrayList<>(expectedReceivedValues.size()); for (final KeyValue receivedKeyValue : receivedKeyValues) { actualValues.add(receivedKeyValue.value); @@ -146,35 +211,50 @@ public void shouldOnlyReadRecordsWhereEarliestSpecified() throws Exception { Collections.sort(actualValues); Collections.sort(expectedReceivedValues); assertThat(actualValues, equalTo(expectedReceivedValues)); - } + private void commitInvalidOffsets() { + final KafkaConsumer consumer = new KafkaConsumer(TestUtils.consumerConfig( + CLUSTER.bootstrapServers(), + streamsConfiguration.getProperty(StreamsConfig.APPLICATION_ID_CONFIG), + StringDeserializer.class, + StringDeserializer.class)); + + final Map invalidOffsets = new HashMap<>(); + invalidOffsets.put(new TopicPartition(TOPIC_1_2, 0), new OffsetAndMetadata(5, null)); + invalidOffsets.put(new TopicPartition(TOPIC_2_2, 0), new OffsetAndMetadata(5, null)); + invalidOffsets.put(new TopicPartition(TOPIC_A_2, 0), new OffsetAndMetadata(5, null)); + invalidOffsets.put(new TopicPartition(TOPIC_C_2, 0), new OffsetAndMetadata(5, null)); + invalidOffsets.put(new TopicPartition(TOPIC_Y_2, 0), new OffsetAndMetadata(5, null)); + invalidOffsets.put(new TopicPartition(TOPIC_Z_2, 0), new OffsetAndMetadata(5, null)); + + consumer.commitSync(invalidOffsets); + + consumer.close(); + } @Test(expected = TopologyBuilderException.class) public void shouldThrowExceptionOverlappingPattern() throws Exception { final KStreamBuilder builder = new KStreamBuilder(); //NOTE this would realistically get caught when building topology, the test is for completeness - final KStream pattern1Stream = builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-[A-D]")); - final KStream pattern2Stream = builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-[A-D]")); - final KStream namedTopicsStream = builder.stream(TOPIC_Y, TOPIC_Z); + builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-[A-D]_1")); + builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-[A-D]_1")); + builder.stream(TOPIC_Y_1, TOPIC_Z_1); builder.earliestResetTopicsPattern(); - } @Test(expected = TopologyBuilderException.class) public void shouldThrowExceptionOverlappingTopic() throws Exception { final KStreamBuilder builder = new KStreamBuilder(); //NOTE this would realistically get caught when building topology, the test is for completeness - final KStream pattern1Stream = builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-[A-D]")); - final KStream pattern2Stream = builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-\\d]")); - final KStream namedTopicsStream = builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, TOPIC_A, TOPIC_Z); + builder.stream(KStreamBuilder.AutoOffsetReset.EARLIEST, Pattern.compile("topic-[A-D]_1")); + builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, Pattern.compile("topic-\\d_1")); + builder.stream(KStreamBuilder.AutoOffsetReset.LATEST, TOPIC_A_1, TOPIC_Z_1); builder.latestResetTopicsPattern(); - } - @Test public void shouldThrowStreamsExceptionNoResetSpecified() throws Exception { Properties props = new Properties(); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java index eeb9177cd3b32..322dd591d8efc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java @@ -133,12 +133,12 @@ public static void beforeTest() throws Exception { streamsConfig = new Properties(); streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfig.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, CLUSTER.zKConnectString()); streamsConfig.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfig.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); final Properties producerConfig = new Properties(); producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java index b8f91faed0341..0e4cdc7ae0c8d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/QueryableStateIntegrationTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.streams.KafkaStreamsTest; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.KGroupedStream; @@ -41,13 +42,9 @@ import org.apache.kafka.streams.state.ReadOnlyWindowStore; import org.apache.kafka.streams.state.StreamsMetadata; import org.apache.kafka.streams.state.WindowStoreIterator; -import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; -import static org.junit.Assert.fail; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.assertEquals; import org.junit.After; import org.junit.Before; import org.junit.ClassRule; @@ -74,6 +71,9 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; @RunWith(Parameterized.class) public class QueryableStateIntegrationTest { @@ -81,6 +81,7 @@ public class QueryableStateIntegrationTest { @ClassRule public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); + public static final int STREAM_THREE_PARTITIONS = 4; private final MockTime mockTime = CLUSTER.time; private String streamOne = "stream-one"; private String streamTwo = "stream-two"; @@ -91,7 +92,7 @@ public class QueryableStateIntegrationTest { private String outputTopicThree = "output-three"; // sufficiently large window size such that everything falls into 1 window private static final long WINDOW_SIZE = TimeUnit.MILLISECONDS.convert(2, TimeUnit.DAYS); - private static final int NUM_PARTITIONS = 2; + private static final int STREAM_TWO_PARTITIONS = 2; private static final int NUM_REPLICAS = NUM_BROKERS; private Properties streamsConfiguration; private List inputValues; @@ -101,7 +102,7 @@ public class QueryableStateIntegrationTest { private Comparator> stringLongComparator; private static int testNo = 0; - public void createTopics() { + public void createTopics() throws InterruptedException { streamOne = streamOne + "-" + testNo; streamConcurrent = streamConcurrent + "-" + testNo; streamThree = streamThree + "-" + testNo; @@ -111,8 +112,8 @@ public void createTopics() { streamTwo = streamTwo + "-" + testNo; CLUSTER.createTopic(streamOne); CLUSTER.createTopic(streamConcurrent); - CLUSTER.createTopic(streamTwo, NUM_PARTITIONS, NUM_REPLICAS); - CLUSTER.createTopic(streamThree, 4, 1); + CLUSTER.createTopic(streamTwo, STREAM_TWO_PARTITIONS, NUM_REPLICAS); + CLUSTER.createTopic(streamThree, STREAM_THREE_PARTITIONS, 1); CLUSTER.createTopic(outputTopic); CLUSTER.createTopic(outputTopicConcurrent); CLUSTER.createTopic(outputTopicThree); @@ -128,23 +129,21 @@ public static Object[] data() { } @Before - public void before() throws IOException { + public void before() throws IOException, InterruptedException { testNo++; createTopics(); streamsConfiguration = new Properties(); final String applicationId = "queryable-state-" + testNo; streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); - streamsConfiguration - .put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory("qs-test").getPath()); streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); - streamsConfiguration - .put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); - streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); - + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + // override this to make the rebalances happen quickly stringComparator = new Comparator>() { @@ -328,7 +327,7 @@ public boolean conditionMet() { @Test public void queryOnRebalance() throws Exception { - final int numThreads = NUM_PARTITIONS; + final int numThreads = STREAM_TWO_PARTITIONS; final StreamRunnable[] streamRunnables = new StreamRunnable[numThreads]; final Thread[] streamThreads = new Thread[numThreads]; final int numIterations = 500000; 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 213fffe561332..0ea36ea6dd6cd 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 @@ -32,12 +32,15 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TopologyBuilder; import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.StreamTask; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; +import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockStateStoreSupplier; import org.apache.kafka.test.StreamsTestUtils; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; @@ -53,11 +56,13 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Properties; import java.util.UUID; import java.util.regex.Pattern; import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; @@ -227,6 +232,31 @@ public boolean conditionMet() { streams.close(); } + @Test + public void shouldAddStateStoreToRegexDefinedSource() throws Exception { + + ProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + MockStateStoreSupplier stateStoreSupplier = new MockStateStoreSupplier("testStateStore", false); + + TopologyBuilder builder = new TopologyBuilder() + .addSource("ingest", Pattern.compile("topic-\\d+")) + .addProcessor("my-processor", processorSupplier, "ingest") + .addStateStore(stateStoreSupplier, "my-processor"); + + + final KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration); + streams.start(); + + final Properties producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(), StringSerializer.class, StringSerializer.class); + + IntegrationTestUtils.produceValuesSynchronously(TOPIC_1, Arrays.asList("message for test"), producerConfig, mockTime); + streams.close(); + + Map> stateStoreToSourceTopic = builder.stateStoreNameToSourceTopics(); + + assertThat(stateStoreToSourceTopic.get("testStateStore").get(0), is("topic-1")); + } + @Test public void testShouldReadFromRegexAndNamedTopics() throws Exception { diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java index f1a8f6844e06c..d653d005ef49f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/ResetIntegrationTest.java @@ -23,6 +23,8 @@ import kafka.utils.ZkUtils; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException; +import org.apache.kafka.common.errors.TimeoutException; +import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; import org.apache.kafka.common.security.JaasUtils; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; @@ -44,7 +46,6 @@ import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; -import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; @@ -92,14 +93,6 @@ public class ResetIntegrationTest { private final MockTime mockTime = CLUSTER.time; private final WaitUntilConsumerGroupGotClosed consumerGroupInactive = new WaitUntilConsumerGroupGotClosed(); - @BeforeClass - public static void startKafkaCluster() throws Exception { - CLUSTER.createTopic(INPUT_TOPIC); - CLUSTER.createTopic(OUTPUT_TOPIC); - CLUSTER.createTopic(OUTPUT_TOPIC_2); - CLUSTER.createTopic(OUTPUT_TOPIC_2_RERUN); - } - @AfterClass public static void globalCleanup() { if (adminClient != null) { @@ -123,17 +116,17 @@ public void cleanup() throws Exception { try { TestUtils.waitForCondition(consumerGroupInactive, TIMEOUT_MULTIPLIER * CLEANUP_CONSUMER_TIMEOUT, "Test consumer group active even after waiting " + (TIMEOUT_MULTIPLIER * CLEANUP_CONSUMER_TIMEOUT) + " ms."); - } catch (GroupCoordinatorNotAvailableException e) { + } catch (final GroupCoordinatorNotAvailableException e) { continue; - } catch (IllegalArgumentException e) { + } catch (final IllegalArgumentException e) { + continue; + } catch (final TimeoutException e) { continue; } break; } - if (testNo == 1) { - prepareInputData(); - } + prepareInputData(); } @Test @@ -280,6 +273,7 @@ private Properties prepareTest() throws Exception { streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 4); streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); streamsConfiguration.put(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, 100); streamsConfiguration.put(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "" + STREAMS_CONSUMER_TIMEOUT); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -290,6 +284,34 @@ private Properties prepareTest() throws Exception { } private void prepareInputData() throws Exception { + try { + CLUSTER.deleteTopic(INPUT_TOPIC); + } catch (final UnknownTopicOrPartitionException e) { + // ignore + } + try { + CLUSTER.deleteTopic(OUTPUT_TOPIC); + } catch (final UnknownTopicOrPartitionException e) { + // ignore + } + try { + CLUSTER.deleteTopic(OUTPUT_TOPIC_2); + } catch (final UnknownTopicOrPartitionException e) { + // ignore + } + try { + CLUSTER.deleteTopic(OUTPUT_TOPIC_2_RERUN); + } catch (final UnknownTopicOrPartitionException e) { + // ignore + } + + waitUntilUserTopicsAreDeleted(); + + CLUSTER.createTopic(INPUT_TOPIC); + CLUSTER.createTopic(OUTPUT_TOPIC); + CLUSTER.createTopic(OUTPUT_TOPIC_2); + CLUSTER.createTopic(OUTPUT_TOPIC_2_RERUN); + final Properties producerConfig = TestUtils.producerConfig(CLUSTER.bootstrapServers(), LongSerializer.class, StringSerializer.class); mockTime.sleep(10); @@ -387,6 +409,34 @@ private void cleanGlobal(final String intermediateUserTopic) { Assert.assertEquals(0, exitCode); } + private void waitUntilUserTopicsAreDeleted() { + ZkUtils zkUtils = null; + try { + zkUtils = ZkUtils.apply(CLUSTER.zKConnectString(), + 30000, + 30000, + JaasUtils.isZkSecurityEnabled()); + + while (userTopicExists(new HashSet<>(scala.collection.JavaConversions.seqAsJavaList(zkUtils.getAllTopics())))) { + Utils.sleep(100); + } + } finally { + if (zkUtils != null) { + zkUtils.close(); + } + } + } + + private boolean userTopicExists(final Set allTopics) { + final Set expectedMissingTopics = new HashSet<>(); + expectedMissingTopics.add(INPUT_TOPIC); + expectedMissingTopics.add(OUTPUT_TOPIC); + expectedMissingTopics.add(OUTPUT_TOPIC_2); + expectedMissingTopics.add(OUTPUT_TOPIC_2_RERUN); + + return expectedMissingTopics.removeAll(allTopics); + } + private void assertInternalTopicsGotDeleted(final String intermediateUserTopic) { final Set expectedRemainingTopicsAfterCleanup = new HashSet<>(); expectedRemainingTopicsAfterCleanup.add(INPUT_TOPIC); 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 519b1f5cb14b6..fe7bebc5635a6 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 @@ -18,13 +18,17 @@ package org.apache.kafka.streams.integration.utils; import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; import kafka.utils.MockTime; import kafka.zk.EmbeddedZookeeper; +import org.apache.kafka.common.TopicPartition; import org.junit.rules.ExternalResource; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import java.util.Properties; /** @@ -34,6 +38,7 @@ public class EmbeddedKafkaCluster extends ExternalResource { private static final Logger log = LoggerFactory.getLogger(EmbeddedKafkaCluster.class); private static final int DEFAULT_BROKER_PORT = 0; // 0 results in a random port being selected + public static final int TOPIC_CREATION_TIMEOUT = 30000; private EmbeddedZookeeper zookeeper = null; private final KafkaEmbedded[] brokers; private final Properties brokerConfig; @@ -63,7 +68,7 @@ public void start() throws IOException, InterruptedException { putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.DeleteTopicEnableProp(), true); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.LogCleanerDedupeBufferSizeProp(), 2 * 1024 * 1024L); putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.GroupMinSessionTimeoutMsProp(), 0); - putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), false); + putIfAbsent(brokerConfig, KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); for (int i = 0; i < brokers.length; i++) { brokerConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), i); @@ -122,7 +127,7 @@ protected void after() { * * @param topic The name of the topic. */ - public void createTopic(final String topic) { + public void createTopic(final String topic) throws InterruptedException { createTopic(topic, 1, 1, new Properties()); } @@ -133,7 +138,7 @@ public void createTopic(final String topic) { * @param partitions The number of partitions for this topic. * @param replication The replication factor for (the partitions of) this topic. */ - public void createTopic(final String topic, final int partitions, final int replication) { + public void createTopic(final String topic, final int partitions, final int replication) throws InterruptedException { createTopic(topic, partitions, replication, new Properties()); } @@ -148,11 +153,24 @@ public void createTopic(final String topic, final int partitions, final int repl public void createTopic(final String topic, final int partitions, final int replication, - final Properties topicConfig) { + final Properties topicConfig) throws InterruptedException { brokers[0].createTopic(topic, partitions, replication, topicConfig); + final List topicPartitions = new ArrayList<>(); + for (int partition = 0; partition < partitions; partition++) { + topicPartitions.add(new TopicPartition(topic, partition)); + } + IntegrationTestUtils.waitForTopicPartitions(brokers(), topicPartitions, TOPIC_CREATION_TIMEOUT); } public void deleteTopic(final String topic) { brokers[0].deleteTopic(topic); } + + public List brokers() { + final List servers = new ArrayList<>(); + for (final KafkaEmbedded broker : brokers) { + servers.add(broker.kafkaServer()); + } + return servers; + } } 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 aa358ab3322e6..08e22cca76302 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 @@ -17,6 +17,10 @@ package org.apache.kafka.streams.integration.utils; +import kafka.api.PartitionStateInfo; +import kafka.api.Request; +import kafka.server.KafkaServer; +import kafka.server.MetadataCache; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; import org.apache.kafka.clients.consumer.KafkaConsumer; @@ -24,12 +28,14 @@ import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; +import scala.Option; import java.io.File; import java.io.IOException; @@ -208,7 +214,7 @@ public boolean conditionMet() { } }; - final String conditionDetails = "Did not receive " + expectedNumRecords + " number of records"; + final String conditionDetails = "Expecting " + expectedNumRecords + " records from topic " + topic + " while only received " + accumData.size() + ": " + accumData; TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails); @@ -248,11 +254,48 @@ public boolean conditionMet() { } }; - final String conditionDetails = "Did not receive " + expectedNumRecords + " number of records"; + final String conditionDetails = "Expecting " + expectedNumRecords + " records from topic " + topic + " while only received " + accumData.size() + ": " + accumData; TestUtils.waitForCondition(valuesRead, waitTime, conditionDetails); return accumData; } + public static void waitForTopicPartitions(final List servers, + final List partitions, + final long timeout) throws InterruptedException { + final long end = System.currentTimeMillis() + timeout; + for (final TopicPartition partition : partitions) { + final long remaining = end - System.currentTimeMillis(); + if (remaining <= 0) { + throw new AssertionError("timed out while waiting for partitions to become available. Timeout=" + timeout); + } + waitUntilMetadataIsPropagated(servers, partition.topic(), partition.partition(), remaining); + } + } + + public static void waitUntilMetadataIsPropagated(final List servers, + final String topic, + final int partition, + final long timeout) throws InterruptedException { + TestUtils.waitForCondition(new TestCondition() { + @Override + public boolean conditionMet() { + for (final KafkaServer server : servers) { + final MetadataCache metadataCache = server.apis().metadataCache(); + final Option partitionInfo = + metadataCache.getPartitionInfo(topic, partition); + if (partitionInfo.isEmpty()) { + return false; + } + final PartitionStateInfo partitionStateInfo = partitionInfo.get(); + if (!Request.isValidBrokerId(partitionStateInfo.leaderIsrAndControllerEpoch().leaderAndIsr().leader())) { + return false; + } + } + return true; + } + }, timeout, "metatadata for topic=" + topic + " partition=" + partition + " not propogated to all brokers"); + + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java index 70c50631deedd..9b48272343e4a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -202,4 +202,7 @@ public void deleteTopic(final String topic) { zkClient.close(); } + public KafkaServer kafkaServer() { + return kafka; + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java index b37e5e8d87986..24387ad687805 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/JoinWindowsTest.java @@ -23,6 +23,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; public class JoinWindowsTest { @@ -44,8 +45,8 @@ public void shouldHaveSaneEqualsAndHashCode() { assertEquals(w2, w1); assertEquals(w1.hashCode(), w2.hashCode()); - JoinWindows w3 = JoinWindows.of(w2.after).before(anyOtherSize); - JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.after); + JoinWindows w3 = JoinWindows.of(w2.afterMs).before(anyOtherSize); + JoinWindows w4 = JoinWindows.of(anyOtherSize).after(w2.afterMs); assertEquals(w3, w4); assertEquals(w4, w3); assertEquals(w3.hashCode(), w4.hashCode()); @@ -55,13 +56,13 @@ public void shouldHaveSaneEqualsAndHashCode() { assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1); assertNotEquals("must be false for different types", new Object(), w1); - JoinWindows differentWindowSize = JoinWindows.of(w1.after + 1); + JoinWindows differentWindowSize = JoinWindows.of(w1.afterMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize, w1); - JoinWindows differentWindowSize2 = JoinWindows.of(w1.after).after(w1.after + 1); + JoinWindows differentWindowSize2 = JoinWindows.of(w1.afterMs).after(w1.afterMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize2, w1); - JoinWindows differentWindowSize3 = JoinWindows.of(w1.after).before(w1.before + 1); + JoinWindows differentWindowSize3 = JoinWindows.of(w1.afterMs).before(w1.beforeMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize3, w1); } @@ -85,14 +86,55 @@ public void timeDifferenceMustNotBeNegative() { JoinWindows.of(-1); } - @Test(expected = IllegalArgumentException.class) - public void afterBelowLower() { - JoinWindows.of(anySize).after(-anySize - 1); + @Test + public void endTimeShouldNotBeBeforeStart() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + try { + windowSpec.after(-anySize - 1); + fail("window end time should not be before window start time"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void beforeOverUpper() { - JoinWindows.of(anySize).before(-anySize - 1); + @Test + public void startTimeShouldNotBeAfterEnd() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + try { + windowSpec.before(-anySize - 1); + fail("window start time should not be after window end time"); + } catch (final IllegalArgumentException e) { + // expected + } + } + + @Test + public void untilShouldSetMaintainDuration() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + final long windowSize = windowSpec.size(); + assertEquals(windowSize, windowSpec.until(windowSize).maintainMs()); + } + + @Test + public void shouldUseWindowSizeForMaintainDurationWhenSizeLargerThanDefaultMaintainMs() { + final long size = Windows.DEFAULT_MAINTAIN_DURATION_MS; + + final JoinWindows windowSpec = JoinWindows.of(size); + final long windowSize = windowSpec.size(); + + assertEquals(windowSize, windowSpec.maintainMs()); + } + + @Test + public void retentionTimeMustNoBeSmallerThanWindowSize() { + final JoinWindows windowSpec = JoinWindows.of(anySize); + final long windowSize = windowSpec.size(); + try { + windowSpec.until(windowSize - 1); + fail("should not accept retention time smaller than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java index c32082c641c96..a469f25fbe754 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -28,6 +28,7 @@ import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockValueJoiner; import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.util.HashSet; @@ -41,8 +42,17 @@ public class KStreamBuilderTest { + private static final String APP_ID = "app-id"; + + private final KStreamBuilder builder = new KStreamBuilder(); + private KStreamTestDriver driver = null; + @Before + public void setUp() { + builder.setApplicationId(APP_ID); + } + @After public void cleanup() { if (driver != null) { @@ -53,8 +63,6 @@ public void cleanup() { @Test(expected = TopologyBuilderException.class) public void testFrom() { - final KStreamBuilder builder = new KStreamBuilder(); - builder.stream("topic-1", "topic-2"); builder.addSource(KStreamImpl.SOURCE_NAME + "0000000000", "topic-3"); @@ -62,17 +70,15 @@ public void testFrom() { @Test public void testNewName() { - KStreamBuilder builder = new KStreamBuilder(); - assertEquals("X-0000000000", builder.newName("X-")); assertEquals("Y-0000000001", builder.newName("Y-")); assertEquals("Z-0000000002", builder.newName("Z-")); - builder = new KStreamBuilder(); + KStreamBuilder newBuilder = new KStreamBuilder(); - assertEquals("X-0000000000", builder.newName("X-")); - assertEquals("Y-0000000001", builder.newName("Y-")); - assertEquals("Z-0000000002", builder.newName("Z-")); + assertEquals("X-0000000000", newBuilder.newName("X-")); + assertEquals("Y-0000000001", newBuilder.newName("Y-")); + assertEquals("Z-0000000002", newBuilder.newName("Z-")); } @Test @@ -80,8 +86,6 @@ public void testMerge() { String topic1 = "topic-1"; String topic2 = "topic-2"; - KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.stream(topic1); KStream source2 = builder.stream(topic2); KStream merged = builder.merge(source1, source2); @@ -105,7 +109,6 @@ public void shouldHaveCorrectSourceTopicsForTableFromMergedStream() throws Excep final String topic1 = "topic-1"; final String topic2 = "topic-2"; final String topic3 = "topic-3"; - final KStreamBuilder builder = new KStreamBuilder(); final KStream source1 = builder.stream(topic1); final KStream source2 = builder.stream(topic2); final KStream source3 = builder.stream(topic3); @@ -130,29 +133,26 @@ public boolean test(final String key, final String value) { final KStream merged = builder.merge(processedSource1, processedSource2, source3); merged.groupByKey().count("my-table"); - final Map> actual = builder.stateStoreNameToSourceTopics(); - assertEquals(Utils.mkSet("topic-1", "topic-2", "topic-3"), actual.get("my-table")); + final Map> actual = builder.stateStoreNameToSourceTopics(); + assertEquals(Utils.mkList("topic-1", "topic-2", "topic-3"), actual.get("my-table")); } @Test(expected = TopologyBuilderException.class) public void shouldThrowExceptionWhenNoTopicPresent() throws Exception { - new KStreamBuilder().stream(); + builder.stream(); } @Test(expected = NullPointerException.class) public void shouldThrowExceptionWhenTopicNamesAreNull() throws Exception { - new KStreamBuilder().stream(Serdes.String(), Serdes.String(), null, null); + builder.stream(Serdes.String(), Serdes.String(), null, null); } @Test public void shouldNotMaterializeSourceKTableIfStateNameNotSpecified() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - builder.setApplicationId("app-id"); - builder.table("topic1", "table1"); builder.table("topic2", null); - ProcessorTopology topology = builder.build(null); + final ProcessorTopology topology = builder.build(null); assertEquals(1, topology.stateStores().size()); assertEquals("table1", topology.stateStores().get(0).name()); @@ -162,23 +162,26 @@ public void shouldNotMaterializeSourceKTableIfStateNameNotSpecified() throws Exc @Test public void shouldBuildSimpleGlobalTableTopology() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); builder.globalTable("table", "globalTable"); + final ProcessorTopology topology = builder.buildGlobalStateTopology(); final List stateStores = topology.globalStateStores(); - final StateStore store = stateStores.iterator().next(); + assertEquals(1, stateStores.size()); - assertEquals("globalTable", store.name()); + assertEquals("globalTable", stateStores.get(0).name()); } @Test public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); builder.globalTable("table", "globalTable"); builder.globalTable("table2", "globalTable2"); + final ProcessorTopology topology = builder.buildGlobalStateTopology(); + final List stateStores = topology.globalStateStores(); - assertEquals(Utils.mkSet("table", "table2"), topology.sourceTopics()); + final Set sourceTopics = topology.sourceTopics(); + + assertEquals(Utils.mkSet("table", "table2"), sourceTopics); assertEquals(2, stateStores.size()); } @@ -186,7 +189,6 @@ public void shouldBuildGlobalTopologyWithAllGlobalTables() throws Exception { public void shouldAddGlobalTablesToEachGroup() throws Exception { final String one = "globalTable"; final String two = "globalTable2"; - final KStreamBuilder builder = new KStreamBuilder(); final GlobalKTable globalTable = builder.globalTable("table", one); final GlobalKTable globalTable2 = builder.globalTable("table2", two); @@ -203,7 +205,7 @@ public String apply(final String key, final String value) { stream.leftJoin(globalTable, kvMapper, MockValueJoiner.TOSTRING_JOINER); final KStream stream2 = builder.stream("t2"); stream2.leftJoin(globalTable2, kvMapper, MockValueJoiner.TOSTRING_JOINER); - builder.setApplicationId("app-id"); + final Map> nodeGroups = builder.nodeGroups(); for (Integer groupId : nodeGroups.keySet()) { final ProcessorTopology topology = builder.build(groupId); @@ -212,6 +214,7 @@ public String apply(final String key, final String value) { for (StateStore stateStore : stateStores) { names.add(stateStore.name()); } + assertEquals(2, stateStores.size()); assertTrue(names.contains(one)); assertTrue(names.contains(two)); @@ -220,17 +223,14 @@ public String apply(final String key, final String value) { @Test public void shouldMapStateStoresToCorrectSourceTopics() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - builder.setApplicationId("app-id"); - final KStream playEvents = builder.stream("events"); final KTable table = builder.table("table-topic", "table-store"); - assertEquals(Collections.singleton("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); + assertEquals(Collections.singletonList("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); final KStream mapped = playEvents.map(MockKeyValueMapper.SelectValueKeyValueMapper()); mapped.leftJoin(table, MockValueJoiner.TOSTRING_JOINER).groupByKey().count("count"); - assertEquals(Collections.singleton("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); - assertEquals(Collections.singleton("app-id-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); + assertEquals(Collections.singletonList("table-topic"), builder.stateStoreNameToSourceTopics().get("table-store")); + assertEquals(Collections.singletonList(APP_ID + "-KSTREAM-MAP-0000000003-repartition"), builder.stateStoreNameToSourceTopics().get("count")); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java new file mode 100644 index 0000000000000..a9eced46cf1a3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/SessionWindowsTest.java @@ -0,0 +1,68 @@ +/** + * 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.kstream; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +public class SessionWindowsTest { + + @Test + public void shouldSetWindowGap() { + final long anyGap = 42L; + assertEquals(anyGap, SessionWindows.with(anyGap).inactivityGap()); + } + + @Test + public void shouldSetWindowRetentionTime() { + final long anyRetentionTime = 42L; + assertEquals(anyRetentionTime, SessionWindows.with(1).until(anyRetentionTime).maintainMs()); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeNegative() { + SessionWindows.with(-1); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeZero() { + SessionWindows.with(0); + } + + @Test + public void retentionTimeShouldBeGapIfGapIsLargerThanDefaultRetentionTime() { + final long windowGap = 2 * Windows.DEFAULT_MAINTAIN_DURATION_MS; + assertEquals(windowGap, SessionWindows.with(windowGap).maintainMs()); + } + + @Test + public void retentionTimeMustNotBeNegative() { + final SessionWindows windowSpec = SessionWindows.with(42); + try { + windowSpec.until(41); + fail("should not accept retention time smaller than gap"); + } catch (final IllegalArgumentException e) { + // expected + } + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java index 2bea16b23b367..6b8b6ea9c99c1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java @@ -26,15 +26,38 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.fail; public class TimeWindowsTest { - private static long anySize = 123L; + private static final long ANY_SIZE = 123L; + + @Test + public void shouldSetWindowSize() { + assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).sizeMs); + } + + @Test + public void shouldSetWindowAdvance() { + final long anyAdvance = 4; + assertEquals(anyAdvance, TimeWindows.of(ANY_SIZE).advanceBy(anyAdvance).advanceMs); + } + + @Test + public void shouldSetWindowRetentionTime() { + assertEquals(ANY_SIZE, TimeWindows.of(ANY_SIZE).until(ANY_SIZE).maintainMs()); + } + + @Test + public void shouldUseWindowSizeAsRentitionTimeIfWindowSizeIsLargerThanDefaultRetentionTime() { + final long windowSize = 2 * Windows.DEFAULT_MAINTAIN_DURATION_MS; + assertEquals(windowSize, TimeWindows.of(windowSize).maintainMs()); + } @Test public void shouldHaveSaneEqualsAndHashCode() { - TimeWindows w1 = TimeWindows.of(anySize); - TimeWindows w2 = TimeWindows.of(w1.size); + TimeWindows w1 = TimeWindows.of(ANY_SIZE); + TimeWindows w2 = TimeWindows.of(w1.sizeMs); // Reflexive assertEquals(w1, w1); @@ -46,7 +69,7 @@ public void shouldHaveSaneEqualsAndHashCode() { assertEquals(w1.hashCode(), w2.hashCode()); // Transitive - TimeWindows w3 = TimeWindows.of(w2.size); + TimeWindows w3 = TimeWindows.of(w2.sizeMs); assertEquals(w2, w3); assertEquals(w1, w3); assertEquals(w1.hashCode(), w3.hashCode()); @@ -56,42 +79,69 @@ public void shouldHaveSaneEqualsAndHashCode() { assertNotEquals("must be false for different window types", UnlimitedWindows.of(), w1); assertNotEquals("must be false for different types", new Object(), w1); - TimeWindows differentWindowSize = TimeWindows.of(w1.size + 1); + TimeWindows differentWindowSize = TimeWindows.of(w1.sizeMs + 1); assertNotEquals("must be false when window sizes are different", differentWindowSize, w1); - TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advance - 1); + TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advanceMs - 1); assertNotEquals("must be false when advance intervals are different", differentAdvanceInterval, w1); } + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeZero() { + TimeWindows.of(0); + } @Test(expected = IllegalArgumentException.class) public void windowSizeMustNotBeNegative() { TimeWindows.of(-1); } - @Test(expected = IllegalArgumentException.class) - public void windowSizeMustNotBeZero() { - TimeWindows.of(0); + @Test + public void advanceIntervalMustNotBeZero() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(0); + fail("should not accept zero advance parameter"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) + @Test public void advanceIntervalMustNotBeNegative() { - TimeWindows.of(anySize).advanceBy(-1); + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(-1); + fail("should not accept negative advance parameter"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void advanceIntervalMustNotBeZero() { - TimeWindows.of(anySize).advanceBy(0); + @Test + public void advanceIntervalMustNotBeLargerThanWindowSize() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.advanceBy(ANY_SIZE + 1); + fail("should not accept advance greater than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } - @Test(expected = IllegalArgumentException.class) - public void advanceIntervalMustNotBeLargerThanWindowSize() { - long size = anySize; - TimeWindows.of(size).advanceBy(size + 1); + @Test + public void retentionTimeMustNoBeSmallerThanWindowSize() { + final TimeWindows windowSpec = TimeWindows.of(ANY_SIZE); + try { + windowSpec.until(ANY_SIZE - 1); + fail("should not accept retention time smaller than window size"); + } catch (final IllegalArgumentException e) { + // expected + } } @Test - public void windowsForHoppingWindows() { + public void shouldComputeWindowsForHoppingWindows() { TimeWindows windows = TimeWindows.of(12L).advanceBy(5L); Map matched = windows.windowsFor(21L); assertEquals(12L / 5L + 1, matched.size()); @@ -101,7 +151,7 @@ public void windowsForHoppingWindows() { } @Test - public void windowsForBarelyOverlappingHoppingWindows() { + public void shouldComputeWindowsForBarelyOverlappingHoppingWindows() { TimeWindows windows = TimeWindows.of(6L).advanceBy(5L); Map matched = windows.windowsFor(7L); assertEquals(1, matched.size()); @@ -109,7 +159,7 @@ public void windowsForBarelyOverlappingHoppingWindows() { } @Test - public void windowsForTumblingWindows() { + public void shouldComputeWindowsForTumblingWindows() { TimeWindows windows = TimeWindows.of(12L); Map matched = windows.windowsFor(21L); assertEquals(1, matched.size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java index c1f4be6246353..ea9078c9dbaca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -26,25 +26,37 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; public class UnlimitedWindowsTest { private static long anyStartTime = 10L; + @Test + public void shouldSetWindowStartTime() { + assertEquals(anyStartTime, UnlimitedWindows.of().startOn(anyStartTime).startMs); + } + @Test(expected = IllegalArgumentException.class) public void startTimeMustNotBeNegative() { UnlimitedWindows.of().startOn(-1); } @Test - public void startTimeCanBeZero() { - UnlimitedWindows.of().startOn(0); + public void shouldThrowOnUntil() { + final UnlimitedWindows windowSpec = UnlimitedWindows.of(); + try { + windowSpec.until(42); + fail("should not allow to set window retention time"); + } catch (final IllegalArgumentException e) { + // expected + } } @Test public void shouldIncludeRecordsThatHappenedOnWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - Map matchedWindows = w.windowsFor(w.start); + Map matchedWindows = w.windowsFor(w.startMs); assertEquals(1, matchedWindows.size()); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); } @@ -52,7 +64,7 @@ public void shouldIncludeRecordsThatHappenedOnWindowStart() { @Test public void shouldIncludeRecordsThatHappenedAfterWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - long timestamp = w.start + 1; + long timestamp = w.startMs + 1; Map matchedWindows = w.windowsFor(timestamp); assertEquals(1, matchedWindows.size()); assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); @@ -61,7 +73,7 @@ public void shouldIncludeRecordsThatHappenedAfterWindowStart() { @Test public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { UnlimitedWindows w = UnlimitedWindows.of().startOn(anyStartTime); - long timestamp = w.start - 1; + long timestamp = w.startMs - 1; Map matchedWindows = w.windowsFor(timestamp); assertTrue(matchedWindows.isEmpty()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java new file mode 100644 index 0000000000000..55c5c60d430a9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowTest.java @@ -0,0 +1,85 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

      + * http://www.apache.org/licenses/LICENSE-2.0 + *

      + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.kafka.streams.kstream; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +public class WindowTest { + + static class TestWindow extends Window { + TestWindow(final long startMs, final long endMs) { + super(startMs, endMs); + } + + @Override + public boolean overlap(final Window other) { + return false; + } + } + + static class TestWindow2 extends Window { + TestWindow2(final long startMs, final long endMs) { + super(startMs, endMs); + } + + @Override + public boolean overlap(final Window other) { + return false; + } + } + + private final TestWindow window = new TestWindow(5, 10); + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfStartIsNegative() { + new TestWindow(-1, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void shouldThrowIfEndIsSmallerThanStart() { + new TestWindow(1, 0); + } + + @Test + public void shouldBeEqualIfStartAndEndSame() { + final TestWindow window2 = new TestWindow(window.startMs, window.endMs); + + assertEquals(window, window); + assertEquals(window, window2); + assertEquals(window2, window); + } + + @Test + public void shouldNotBeEqualIfStartOrEndIsDifferent() { + assertNotEquals(window, new TestWindow(0, window.endMs)); + assertNotEquals(window, new TestWindow(7, window.endMs)); + assertNotEquals(window, new TestWindow(window.startMs, 7)); + assertNotEquals(window, new TestWindow(window.startMs, 15)); + assertNotEquals(window, new TestWindow(7, 8)); + assertNotEquals(window, new TestWindow(0, 15)); + } + + @Test + public void shouldNotBeEqualIfDifferentWindowType() { + assertNotEquals(window, new TestWindow2(window.startMs, window.endMs)); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java new file mode 100644 index 0000000000000..890265f917776 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/WindowsTest.java @@ -0,0 +1,62 @@ +/** + * 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.kstream; + +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class WindowsTest { + + private class TestWindows extends Windows { + + @Override + public Map windowsFor(long timestamp) { + return null; + } + + @Override + public long size() { + return 0; + } + } + + @Test + public void shouldSetNumberOfSegments() { + final int anySegmentSizeLargerThanOne = 5; + assertEquals(anySegmentSizeLargerThanOne, new TestWindows().segments(anySegmentSizeLargerThanOne).segments); + } + + @Test + public void shouldSetWindowRetentionTime() { + final int anyNotNegativeRetentionTime = 42; + assertEquals(anyNotNegativeRetentionTime, new TestWindows().until(anyNotNegativeRetentionTime).maintainMs()); + } + + @Test(expected = IllegalArgumentException.class) + public void numberOfSegmentsMustBeAtLeastTwo() { + new TestWindows().segments(1); + } + + @Test(expected = IllegalArgumentException.class) + public void retentionTimeMustNotBeNegative() { + new TestWindows().until(-1); + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java index bbc9741ca2be6..c41d9539f597e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/GlobalKTableJoinsTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockValueJoiner; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -38,14 +39,15 @@ public class GlobalKTableJoinsTest { private final KStreamBuilder builder = new KStreamBuilder(); - private GlobalKTable global; - private File stateDir; private final Map results = new HashMap<>(); + private final String streamTopic = "stream"; + private final String globalTopic = "global"; + private File stateDir; + private GlobalKTable global; private KStream stream; private KeyValueMapper keyValueMapper; private ForeachAction action; - private final String streamTopic = "stream"; - private final String globalTopic = "global"; + private KStreamTestDriver driver = null; @Before public void setUp() throws Exception { @@ -64,7 +66,14 @@ public void apply(final String key, final String value) { results.put(key, value); } }; + } + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; } @Test @@ -94,7 +103,7 @@ public void shouldInnerJoinWithStream() throws Exception { } private void verifyJoin(final Map expected, final String joinInput) { - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir); + driver = new KStreamTestDriver(builder, stateDir); driver.setTime(0L); // write some data to the global table driver.process(globalTopic, "a", "A"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java index 729e1905b43d3..3fd287d7304b4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedStreamImplTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; @@ -31,18 +32,25 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.kstream.Windows; +import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Before; import org.junit.Test; +import java.util.ArrayList; +import java.util.Arrays; import java.util.HashMap; +import java.util.List; import java.util.Map; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; public class KGroupedStreamImplTest { @@ -50,6 +58,7 @@ public class KGroupedStreamImplTest { private static final String TOPIC = "topic"; private final KStreamBuilder builder = new KStreamBuilder(); private KGroupedStream groupedStream; + private KStreamTestDriver driver = null; @Before public void before() { @@ -57,6 +66,14 @@ public void before() { groupedStream = stream.groupByKey(Serdes.String(), Serdes.String()); } + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test(expected = NullPointerException.class) public void shouldNotHaveNullReducerOnReduce() throws Exception { groupedStream.reduce(null, "store"); @@ -64,14 +81,12 @@ public void shouldNotHaveNullReducerOnReduce() throws Exception { @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreNameOnReduce() throws Exception { - String storeName = null; - groupedStream.reduce(MockReducer.STRING_ADDER, storeName); + groupedStream.reduce(MockReducer.STRING_ADDER, (String) null); } @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreSupplierOnReduce() throws Exception { - StateStoreSupplier storeSupplier = null; - groupedStream.reduce(MockReducer.STRING_ADDER, storeSupplier); + groupedStream.reduce(MockReducer.STRING_ADDER, (StateStoreSupplier) null); } @Test(expected = NullPointerException.class) @@ -86,8 +101,7 @@ public void shouldNotHaveNullWindowsWithWindowedReduce() throws Exception { @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreNameWithWindowedReduce() throws Exception { - String storeName = null; - groupedStream.reduce(MockReducer.STRING_ADDER, TimeWindows.of(10), storeName); + groupedStream.reduce(MockReducer.STRING_ADDER, TimeWindows.of(10), (String) null); } @Test(expected = NullPointerException.class) @@ -102,8 +116,7 @@ public void shouldNotHaveNullAdderOnAggregate() throws Exception { @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreNameOnAggregate() throws Exception { - String storeName = null; - groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Serdes.String(), storeName); + groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, Serdes.String(), null); } @Test(expected = NullPointerException.class) @@ -123,14 +136,12 @@ public void shouldNotHaveNullWindowsOnWindowedAggregate() throws Exception { @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreNameOnWindowedAggregate() throws Exception { - String storeName = null; - groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), Serdes.String(), storeName); + groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), Serdes.String(), null); } @Test(expected = NullPointerException.class) public void shouldNotHaveNullStoreSupplierOnWindowedAggregate() throws Exception { - StateStoreSupplier storeSupplier = null; - groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), storeSupplier); + groupedStream.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, TimeWindows.of(10), null); } @Test @@ -159,7 +170,7 @@ public void apply(final Windowed key, final Integer value) { } }); - final KStreamTestDriver driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); + driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); driver.setTime(10); driver.process(TOPIC, "1", "1"); driver.setTime(15); @@ -188,7 +199,7 @@ public void apply(final Windowed key, final Long value) { results.put(key, value); } }); - final KStreamTestDriver driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); + driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); driver.setTime(10); driver.process(TOPIC, "1", "1"); driver.setTime(15); @@ -224,7 +235,7 @@ public void apply(final Windowed key, final String value) { results.put(key, value); } }); - final KStreamTestDriver driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); + driver = new KStreamTestDriver(builder, TestUtils.tempDirectory()); driver.setTime(10); driver.process(TOPIC, "1", "A"); driver.setTime(15); @@ -337,4 +348,38 @@ public void shouldNotAcceptNullStoreNameWhenCountingSessionWindows() throws Exce public void shouldNotAcceptNullStoreStoreSupplierNameWhenCountingSessionWindows() throws Exception { groupedStream.count(SessionWindows.with(90), (StateStoreSupplier) null); } -} + + @Test + public void shouldCountWindowed() throws Exception { + final List, Long>> results = new ArrayList<>(); + groupedStream.count( + TimeWindows.of(500L), + "aggregate-by-key-windowed") + .foreach(new ForeachAction, Long>() { + @Override + public void apply(final Windowed key, final Long value) { + results.add(KeyValue.pair(key, value)); + } + }); + + driver = new KStreamTestDriver(builder, TestUtils.tempDirectory(), 0); + driver.setTime(0); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "2", "B"); + driver.process(TOPIC, "3", "C"); + driver.setTime(500); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "1", "A"); + driver.process(TOPIC, "2", "B"); + driver.process(TOPIC, "2", "B"); + assertThat(results, equalTo(Arrays.asList( + KeyValue.pair(new Windowed<>("1", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("3", new TimeWindow(0, 500)), 1L), + KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 1L), + KeyValue.pair(new Windowed<>("1", new TimeWindow(500, 1000)), 2L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 1L), + KeyValue.pair(new Windowed<>("2", new TimeWindow(500, 1000)), 2L) + ))); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 5ed61e1c750e2..5fded02dec1bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -24,13 +24,13 @@ import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockAggregator; import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockReducer; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -42,19 +42,27 @@ public class KGroupedTableImplTest { + private final KStreamBuilder builder = new KStreamBuilder(); private KGroupedTable groupedTable; + private KStreamTestDriver driver = null; @Before public void before() { - final KStreamBuilder builder = new KStreamBuilder(); groupedTable = builder.table(Serdes.String(), Serdes.String(), "blah", "blah") .groupBy(MockKeyValueMapper.SelectValueKeyValueMapper()); } + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test(expected = NullPointerException.class) public void shouldNotAllowNullStoreNameOnAggregate() throws Exception { - String storeName = null; - groupedTable.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, MockAggregator.TOSTRING_REMOVER, storeName); + groupedTable.aggregate(MockInitializer.STRING_INIT, MockAggregator.TOSTRING_ADDER, MockAggregator.TOSTRING_REMOVER, (String) null); } @Test(expected = NullPointerException.class) @@ -84,19 +92,16 @@ public void shouldNotAllowNullSubtractorOnReduce() throws Exception { @Test(expected = NullPointerException.class) public void shouldNotAllowNullStoreNameOnReduce() throws Exception { - String storeName = null; - groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, storeName); + groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, (String) null); } @Test(expected = NullPointerException.class) public void shouldNotAllowNullStoreSupplierOnReduce() throws Exception { - StateStoreSupplier storeName = null; - groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, storeName); + groupedTable.reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, (String) null); } @Test public void shouldReduce() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); final String topic = "input"; final KeyValueMapper> intProjection = new KeyValueMapper>() { @@ -118,8 +123,7 @@ public void apply(final String key, final Integer value) { } }); - - final KStreamTestDriver driver = new KStreamTestDriver(builder, TestUtils.tempDirectory(), Serdes.String(), Serdes.Integer()); + driver = new KStreamTestDriver(builder, TestUtils.tempDirectory(), Serdes.String(), Serdes.Integer()); driver.setTime(10L); driver.process(topic, "A", 1.1); driver.process(topic, "B", 2.2); @@ -136,6 +140,5 @@ public void apply(final String key, final Integer value) { assertEquals(Integer.valueOf(5), results.get("A")); assertEquals(Integer.valueOf(6), results.get("B")); - } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 569ea5af0f921..b6988e31587e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -141,6 +141,4 @@ public void testJoin() throws Exception { processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); } - - } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java index c3368a116a90f..2e5b201186199 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSessionWindowAggregateProcessorTest.java @@ -22,7 +22,6 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; -import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Merger; import org.apache.kafka.streams.kstream.SessionWindows; import org.apache.kafka.streams.kstream.Windowed; @@ -32,7 +31,6 @@ import org.apache.kafka.streams.state.internals.RocksDBSessionStoreSupplier; import org.apache.kafka.streams.state.SessionStore; import org.apache.kafka.streams.state.internals.ThreadCache; -import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.NoOpRecordCollector; import org.apache.kafka.test.TestUtils; @@ -91,7 +89,7 @@ public Long apply(final String aggKey, final Long aggOne, final Long aggTwo) { @Before public void initializeStore() { final File stateDir = TestUtils.tempDirectory(); - context = new MockProcessorContext(new KStreamTestDriver(new KStreamBuilder(), stateDir), stateDir, + context = new MockProcessorContext(stateDir, Serdes.String(), Serdes.String(), new NoOpRecordCollector(), new ThreadCache("testCache", 100000, new MockStreamsMetrics(new Metrics()))) { @Override public void forward(final K key, final V value) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index a43cab629caba..39baa4ebc9ecb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -220,7 +220,7 @@ public void testCount() throws IOException { .toStream() .process(proc); - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir); + driver = new KStreamTestDriver(builder, stateDir); driver.process(input, "A", "green"); driver.flushState(); @@ -256,7 +256,7 @@ public void testCountCoalesced() throws IOException { .toStream() .process(proc); - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir); + driver = new KStreamTestDriver(builder, stateDir); driver.process(input, "A", "green"); driver.process(input, "B", "green"); @@ -309,7 +309,7 @@ public String apply(String key, String value, String aggregate) { .toStream() .process(proc); - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir); + driver = new KStreamTestDriver(builder, stateDir); driver.process(input, "11", "A"); driver.flushState(); @@ -378,7 +378,7 @@ public String apply(final String value) { } }); - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, 111); + driver = new KStreamTestDriver(builder, stateDir, 111); driver.process(reduceTopic, "1", new Change<>(1L, null)); driver.process("tableOne", "2", "2"); // this should trigger eviction on the reducer-store topic diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index 4bb64b87d363b..83cf96518f5d5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -103,6 +103,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic1, null, "SomeVal"); driver.flushState(); processor.checkAndClearProcessResult(); @@ -112,6 +114,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic2, null, "AnotherVal"); driver.flushState(); processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); @@ -119,8 +123,8 @@ public void testJoin() throws Exception { // push all four items to the primary stream. this should produce two items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + for (int expectedKey : expectedKeys) { + driver.process(topic1, expectedKey, "XX" + expectedKey); } driver.flushState(); @@ -128,8 +132,8 @@ public void testJoin() throws Exception { checkJoinedValues(getter, kv(0, "XX0+Y0"), kv(1, "XX1+Y1")); // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + for (int expectedKey : expectedKeys) { + driver.process(topic2, expectedKey, "YY" + expectedKey); } driver.flushState(); @@ -138,8 +142,8 @@ public void testJoin() throws Exception { // push all four items to the primary stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + for (int expectedKey : expectedKeys) { + driver.process(topic1, expectedKey, "X" + expectedKey); } driver.flushState(); @@ -165,6 +169,10 @@ public void testJoin() throws Exception { processor.checkAndClearProcessResult("2:XX2+YY2", "3:XX3+YY3"); checkJoinedValues(getter, kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); + + driver.process(topic1, null, "XX" + 1); + checkJoinedValues(getter, kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); + } @Test @@ -311,8 +319,8 @@ public void testSendingOldValues() throws Exception { // push all four items to the primary stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + for (int expectedKey : expectedKeys) { + driver.process(topic1, expectedKey, "X" + expectedKey); } driver.flushState(); proc.checkAndClearProcessResult("0:(X0+YY0<-XX0+YY0)", "1:(X1+YY1<-XX1+YY1)", "2:(X2+YY2<-XX2+YY2)", "3:(X3+YY3<-XX3+YY3)"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index cbf1da4aed8a7..9483c6ba3ba1c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -104,6 +104,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic1, null, "SomeVal"); driver.flushState(); processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); @@ -114,6 +116,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic2, null, "AnotherVal"); driver.flushState(); processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); @@ -367,7 +371,6 @@ public String apply(final String value) { }; final KTable seven = one.mapValues(mapper); - final KTable eight = six.leftJoin(seven, MockValueJoiner.TOSTRING_JOINER); aggTable.leftJoin(one, MockValueJoiner.TOSTRING_JOINER) @@ -378,7 +381,7 @@ public String apply(final String value) { .leftJoin(eight, MockValueJoiner.TOSTRING_JOINER) .mapValues(mapper); - final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, 250); + driver = new KStreamTestDriver(builder, stateDir, 250); final String[] values = {"a", "AA", "BBB", "CCCC", "DD", "EEEEEEEE", "F", "GGGGGGGGGGGGGGG", "HHH", "IIIIIIIIII", "J", "KK", "LLLL", "MMMMMMMMMMMMMMMMMMMMMM", "NNNNN", "O", "P", "QQQQQ", "R", "SSSS", @@ -387,7 +390,7 @@ public String apply(final String value) { final Random random = new Random(); for (int i = 0; i < 1000; i++) { for (String input : inputs) { - final Long key = Long.valueOf(random.nextInt(1000)); + final Long key = (long) random.nextInt(1000); final String value = values[random.nextInt(values.length)]; driver.process(input, key, value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 02e4e605a4991..65e5595de72dd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -102,6 +102,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic1, null, "SomeVal"); driver.flushState(); processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); @@ -111,6 +113,8 @@ public void testJoin() throws Exception { for (int i = 0; i < 2; i++) { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } + // pass tuple with null key, it will be discarded in join process + driver.process(topic2, null, "AnotherVal"); driver.flushState(); processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java index d1bff05b83d0c..4ab1c35d0b2a9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java @@ -38,11 +38,11 @@ public class KeyValuePrinterProcessorTest { - private String topicName = "topic"; - private Serde stringSerde = Serdes.String(); - private ByteArrayOutputStream baos = new ByteArrayOutputStream(); - private KStreamBuilder builder = new KStreamBuilder(); - private PrintStream printStream = new PrintStream(baos); + private final String topicName = "topic"; + private final Serde stringSerde = Serdes.String(); + private final ByteArrayOutputStream baos = new ByteArrayOutputStream(); + private final KStreamBuilder builder = new KStreamBuilder(); + private final PrintStream printStream = new PrintStream(baos); private KStreamTestDriver driver = null; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java index 3a0f490bf6d0c..5c70c8ad32b9c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/SessionKeySerdeTest.java @@ -58,30 +58,38 @@ public void shouldDeSerializeNullToNull() throws Exception { @Test public void shouldConvertToBinaryAndBack() throws Exception { final Windowed key = new Windowed<>("key", new SessionWindow(10, 20)); - final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer()); - final Windowed result = SessionKeySerde.from(serialized.get(), Serdes.String().deserializer()); + final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer(), "topic"); + final Windowed result = SessionKeySerde.from(serialized.get(), Serdes.String().deserializer(), "topic"); assertEquals(key, result); } @Test public void shouldExtractEndTimeFromBinary() throws Exception { final Windowed key = new Windowed<>("key", new SessionWindow(10, 100)); - final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer()); + final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer(), "topic"); assertEquals(100, SessionKeySerde.extractEnd(serialized.get())); } @Test public void shouldExtractStartTimeFromBinary() throws Exception { final Windowed key = new Windowed<>("key", new SessionWindow(50, 100)); - final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer()); + final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer(), "topic"); assertEquals(50, SessionKeySerde.extractStart(serialized.get())); } @Test public void shouldExtractKeyBytesFromBinary() throws Exception { final Windowed key = new Windowed<>("blah", new SessionWindow(50, 100)); - final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer()); + final Bytes serialized = SessionKeySerde.toBinary(key, Serdes.String().serializer(), "topic"); assertArrayEquals("blah".getBytes(), SessionKeySerde.extractKeyBytes(serialized.get())); } + @Test + public void shouldExtractBytesKeyFromBinary() throws Exception { + final Bytes bytesKey = Bytes.wrap("key".getBytes()); + final Windowed windowedBytesKey = new Windowed<>(bytesKey, new SessionWindow(0, 10)); + final Bytes serialized = SessionKeySerde.bytesToBinary(windowedBytesKey); + assertEquals(windowedBytesKey, SessionKeySerde.fromBytes(serialized)); + } + } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.java new file mode 100644 index 0000000000000..efa20b8be7464 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/TimeWindowTest.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.streams.kstream.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TimeWindowTest { + + private long start = 50; + private long end = 100; + private final TimeWindow window = new TimeWindow(start, end); + private final SessionWindow sessionWindow = new SessionWindow(start, end); + + @Test(expected = IllegalArgumentException.class) + public void endMustBeLargerThanStart() { + new TimeWindow(start, start); + } + + @Test + public void shouldNotOverlapIfOtherWindowIsBeforeThisWindow() { + /* + * This: [-------) + * Other: [-----) + */ + assertFalse(window.overlap(new TimeWindow(0, 25))); + assertFalse(window.overlap(new TimeWindow(0, start - 1))); + assertFalse(window.overlap(new TimeWindow(0, start))); + } + + @Test + public void shouldOverlapIfOtherWindowEndIsWithinThisWindow() { + /* + * This: [-------) + * Other: [---------) + */ + assertTrue(window.overlap(new TimeWindow(0, start + 1))); + assertTrue(window.overlap(new TimeWindow(0, 75))); + assertTrue(window.overlap(new TimeWindow(0, end - 1))); + + assertTrue(window.overlap(new TimeWindow(start - 1, start + 1))); + assertTrue(window.overlap(new TimeWindow(start - 1, 75))); + assertTrue(window.overlap(new TimeWindow(start - 1, end - 1))); + } + + @Test + public void shouldOverlapIfOtherWindowContainsThisWindow() { + /* + * This: [-------) + * Other: [------------------) + */ + assertTrue(window.overlap(new TimeWindow(0, end))); + assertTrue(window.overlap(new TimeWindow(0, end + 1))); + assertTrue(window.overlap(new TimeWindow(0, 150))); + + assertTrue(window.overlap(new TimeWindow(start - 1, end))); + assertTrue(window.overlap(new TimeWindow(start - 1, end + 1))); + assertTrue(window.overlap(new TimeWindow(start - 1, 150))); + + assertTrue(window.overlap(new TimeWindow(start, end))); + assertTrue(window.overlap(new TimeWindow(start, end + 1))); + assertTrue(window.overlap(new TimeWindow(start, 150))); + } + + @Test + public void shouldOverlapIfOtherWindowIsWithinThisWindow() { + /* + * This: [-------) + * Other: [---) + */ + assertTrue(window.overlap(new TimeWindow(start, 75))); + assertTrue(window.overlap(new TimeWindow(start, end))); + assertTrue(window.overlap(new TimeWindow(75, end))); + } + + @Test + public void shouldOverlapIfOtherWindowStartIsWithinThisWindow() { + /* + * This: [-------) + * Other: [-------) + */ + assertTrue(window.overlap(new TimeWindow(start, end + 1))); + assertTrue(window.overlap(new TimeWindow(start, 150))); + assertTrue(window.overlap(new TimeWindow(75, end + 1))); + assertTrue(window.overlap(new TimeWindow(75, 150))); + } + + @Test + public void shouldNotOverlapIsOtherWindowIsAfterThisWindow() { + /* + * This: [-------) + * Other: [------) + */ + assertFalse(window.overlap(new TimeWindow(end, end + 1))); + assertFalse(window.overlap(new TimeWindow(end, 150))); + assertFalse(window.overlap(new TimeWindow(end + 1, 150))); + assertFalse(window.overlap(new TimeWindow(125, 150))); + } + + @Test(expected = IllegalArgumentException.class) + public void cannotCompareTimeWindowWithDifferentWindowType() { + window.overlap(sessionWindow); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java new file mode 100644 index 0000000000000..f3c9cfbff89c9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindowTest.java @@ -0,0 +1,42 @@ +/** + * 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.kstream.internals; + +import org.junit.Test; + +import static org.junit.Assert.assertTrue; + +public class UnlimitedWindowTest { + + private long start = 50; + private final UnlimitedWindow window = new UnlimitedWindow(start); + private final SessionWindow sessionWindow = new SessionWindow(start, start); + + @Test + public void shouldAlwaysOverlap() { + assertTrue(window.overlap(new UnlimitedWindow(start - 1))); + assertTrue(window.overlap(new UnlimitedWindow(start))); + assertTrue(window.overlap(new UnlimitedWindow(start + 1))); + } + + @Test(expected = IllegalArgumentException.class) + public void cannotCompareUnlimitedWindowWithDifferentWindowType() { + window.overlap(sessionWindow); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 4be83bebf2566..efb3fd229d7f1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -70,7 +70,7 @@ public void testCopartitioning() { DefaultPartitioner defaultPartitioner = new DefaultPartitioner(); WindowedSerializer windowedSerializer = new WindowedSerializer<>(intSerializer); - WindowedStreamPartitioner streamPartitioner = new WindowedStreamPartitioner<>(windowedSerializer); + WindowedStreamPartitioner streamPartitioner = new WindowedStreamPartitioner<>(topicName, windowedSerializer); for (int k = 0; k < 10; k++) { Integer key = rand.nextInt(); @@ -81,7 +81,7 @@ public void testCopartitioning() { Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster); - for (int w = 0; w < 10; w++) { + for (int w = 1; w < 10; w++) { TimeWindow window = new TimeWindow(10 * w, 20 * w); Windowed windowedKey = new Windowed<>(key, window); diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java index 7ba61613d43cb..55d9c68031876 100644 --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -14,7 +14,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.perf; import org.apache.kafka.clients.consumer.ConsumerConfig; @@ -46,23 +45,43 @@ import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; +import org.apache.kafka.test.TestUtils; import java.io.File; import java.util.ArrayList; import java.util.List; +import java.util.Locale; import java.util.concurrent.CountDownLatch; import java.util.Properties; import java.util.Random; +import java.util.concurrent.TimeUnit; +/** + * Class that provides support for a series of benchmarks. It is usually driven by + * tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py. + * If ran manually through the main() function below, you must do the following: + * 1. Have ZK and a Kafka broker set up + * 2. Run the loading step first: SimpleBenchmark localhost:9092 /tmp/statedir numRecords true "all" + * 3. Run the stream processing step second: SimpleBenchmark localhost:9092 /tmp/statedir numRecords false "all" + * Note that what changed is the 4th parameter, from "true" indicating that is a load phase, to "false" indicating + * that this is a real run. + * + * Note that "all" is a convenience option when running this test locally and will not work when running the test + * at scale (through tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py). That is due to exact syncronization + * needs for each test (e.g., you wouldn't want one instance to run "count" while another + * is still running "consume" + */ public class SimpleBenchmark { private final String kafka; - private final String zookeeper; private final File stateDir; - + private final Boolean loadPhase; + private final String testName; + private static final String ALL_TESTS = "all"; private static final String SOURCE_TOPIC = "simpleBenchmarkSourceTopic"; private static final String SINK_TOPIC = "simpleBenchmarkSinkTopic"; + private static final String COUNT_TOPIC = "countTopic"; private static final String JOIN_TOPIC_1_PREFIX = "joinSourceTopic1"; private static final String JOIN_TOPIC_2_PREFIX = "joinSourceTopic2"; private static final ValueJoiner VALUE_JOINER = new ValueJoiner() { @@ -83,27 +102,90 @@ public byte[] apply(final byte[] value1, final byte[] value2) { }; private static int numRecords; - private static Integer endKey; - private static final int KEY_SIZE = 8; + private static int processedRecords = 0; + private static long processedBytes = 0; private static final int VALUE_SIZE = 100; - private static final int RECORD_SIZE = KEY_SIZE + VALUE_SIZE; + private static final long POLL_MS = 500L; + private static final int MAX_POLL_RECORDS = 1000; + private static final int SOCKET_SIZE_BYTES = 1 * 1024 * 1024; private static final Serde BYTE_SERDE = Serdes.ByteArray(); private static final Serde INTEGER_SERDE = Serdes.Integer(); - public SimpleBenchmark(File stateDir, String kafka, String zookeeper) { + public SimpleBenchmark(final File stateDir, final String kafka, final Boolean loadPhase, final String testName) { super(); this.stateDir = stateDir; this.kafka = kafka; - this.zookeeper = zookeeper; + this.loadPhase = loadPhase; + this.testName = testName; + } + + private void run() throws Exception { + switch (testName) { + case ALL_TESTS: + // producer performance + produce(SOURCE_TOPIC); + // consumer performance + consume(SOURCE_TOPIC); + // simple stream performance source->process + processStream(SOURCE_TOPIC); + // simple stream performance source->sink + processStreamWithSink(SOURCE_TOPIC); + // simple stream performance source->store + processStreamWithStateStore(SOURCE_TOPIC); + // simple stream performance source->cache->store + processStreamWithCachedStateStore(SOURCE_TOPIC); + // simple aggregation + count(COUNT_TOPIC); + // simple streams performance KSTREAM-KTABLE join + kStreamKTableJoin(JOIN_TOPIC_1_PREFIX + "KStreamKTable", JOIN_TOPIC_2_PREFIX + "KStreamKTable"); + // simple streams performance KSTREAM-KSTREAM join + kStreamKStreamJoin(JOIN_TOPIC_1_PREFIX + "KStreamKStream", JOIN_TOPIC_2_PREFIX + "KStreamKStream"); + // simple streams performance KTABLE-KTABLE join + kTableKTableJoin(JOIN_TOPIC_1_PREFIX + "KTableKTable", JOIN_TOPIC_2_PREFIX + "KTableKTable"); + break; + case "produce": + produce(SOURCE_TOPIC); + break; + case "consume": + consume(SOURCE_TOPIC); + break; + case "count": + count(COUNT_TOPIC); + break; + case "processstream": + processStream(SOURCE_TOPIC); + break; + case "processstreamwithsink": + processStreamWithSink(SOURCE_TOPIC); + break; + case "processstreamwithstatestore": + processStreamWithStateStore(SOURCE_TOPIC); + break; + case "processstreamwithcachedstatestore": + processStreamWithCachedStateStore(SOURCE_TOPIC); + break; + case "kstreamktablejoin": + kStreamKTableJoin(JOIN_TOPIC_1_PREFIX + "KStreamKTable", JOIN_TOPIC_2_PREFIX + "KStreamKTable"); + break; + case "kstreamkstreamjoin": + kStreamKStreamJoin(JOIN_TOPIC_1_PREFIX + "KStreamKStream", JOIN_TOPIC_2_PREFIX + "KStreamKStream"); + break; + case "ktablektablejoin": + kTableKTableJoin(JOIN_TOPIC_1_PREFIX + "KTableKTable", JOIN_TOPIC_2_PREFIX + "KTableKTable"); + break; + default: + throw new Exception("Unknown test name " + testName); + + } } public static void main(String[] args) throws Exception { String kafka = args.length > 0 ? args[0] : "localhost:9092"; - String zookeeper = args.length > 1 ? args[1] : "localhost:2181"; - String stateDirStr = args.length > 2 ? args[2] : "/tmp/kafka-streams-simple-benchmark"; - numRecords = args.length > 3 ? Integer.parseInt(args[3]) : 10000000; - endKey = numRecords - 1; + String stateDirStr = args.length > 1 ? args[1] : TestUtils.tempDirectory().getAbsolutePath(); + numRecords = args.length > 2 ? Integer.parseInt(args[2]) : 10000000; + boolean loadPhase = args.length > 3 ? Boolean.parseBoolean(args[3]) : false; + String testName = args.length > 4 ? args[4].toLowerCase(Locale.ROOT) : ALL_TESTS; final File stateDir = new File(stateDirStr); stateDir.mkdir(); @@ -111,65 +193,121 @@ public static void main(String[] args) throws Exception { rocksdbDir.mkdir(); // Note: this output is needed for automated tests and must not be removed - System.out.println("SimpleBenchmark instance started"); + System.out.println("StreamsTest instance started"); System.out.println("kafka=" + kafka); - System.out.println("zookeeper=" + zookeeper); System.out.println("stateDir=" + stateDir); System.out.println("numRecords=" + numRecords); + System.out.println("loadPhase=" + loadPhase); + System.out.println("testName=" + testName); + + SimpleBenchmark benchmark = new SimpleBenchmark(stateDir, kafka, loadPhase, testName); + benchmark.run(); + } - SimpleBenchmark benchmark = new SimpleBenchmark(stateDir, kafka, zookeeper); - - // producer performance - benchmark.produce(SOURCE_TOPIC, VALUE_SIZE, "simple-benchmark-produce", numRecords, true, numRecords, true); - // consumer performance - benchmark.consume(SOURCE_TOPIC); - // simple stream performance source->process - benchmark.processStream(SOURCE_TOPIC); - // simple stream performance source->sink - benchmark.processStreamWithSink(SOURCE_TOPIC); - // simple stream performance source->store - benchmark.processStreamWithStateStore(SOURCE_TOPIC); - // simple stream performance source->cache->store - benchmark.processStreamWithCachedStateStore(SOURCE_TOPIC); - // simple streams performance KSTREAM-KTABLE join - benchmark.kStreamKTableJoin(JOIN_TOPIC_1_PREFIX + "kStreamKTable", JOIN_TOPIC_2_PREFIX + "kStreamKTable"); - // simple streams performance KSTREAM-KSTREAM join - benchmark.kStreamKStreamJoin(JOIN_TOPIC_1_PREFIX + "kStreamKStream", JOIN_TOPIC_2_PREFIX + "kStreamKStream"); - // simple streams performance KTABLE-KTABLE join - benchmark.kTableKTableJoin(JOIN_TOPIC_1_PREFIX + "kTableKTable", JOIN_TOPIC_2_PREFIX + "kTableKTable"); - } - - private Properties setJoinProperties(final String applicationId) { + private Properties setStreamProperties(final String applicationId) { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, SOCKET_SIZE_BYTES); props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.ByteArray().getClass()); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MAX_POLL_RECORDS); + props.put(StreamsConfig.POLL_MS_CONFIG, POLL_MS); return props; } + private Properties setProduceConsumeProperties(final String clientId) { + Properties props = new Properties(); + props.put(ProducerConfig.CLIENT_ID_CONFIG, clientId); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ProducerConfig.SEND_BUFFER_CONFIG, SOCKET_SIZE_BYTES); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + // the socket buffer needs to be large, especially when running in AWS with + // high latency. if running locally the default is fine. + props.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, SOCKET_SIZE_BYTES); + props.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, MAX_POLL_RECORDS); + return props; + } + + private boolean maybeSetupPhase(final String topic, final String clientId, + final boolean skipIfAllTests) throws Exception { + processedRecords = 0; + processedBytes = 0; + // initialize topics + if (loadPhase) { + if (skipIfAllTests) { + // if we run all tests, the produce test will have already loaded the data + if (testName.equals(ALL_TESTS)) { + // Skipping loading phase since previously loaded + return true; + } + } + System.out.println("Initializing topic " + topic); + // WARNING: The keys must be sequential, i.e., unique, otherwise the logic for when this test + // stops will not work (in createCountStreams) + produce(topic, VALUE_SIZE, clientId, numRecords, true, numRecords, false); + return true; + } + return false; + } + + private KafkaStreams createCountStreams(Properties streamConfig, String topic, final CountDownLatch latch) { + final KStreamBuilder builder = new KStreamBuilder(); + final KStream input = builder.stream(topic); + + input.groupByKey() + .count("tmpStoreName").foreach(new CountDownAction(latch)); + + return new KafkaStreams(builder, streamConfig); + } + + /** + * Measure the performance of a simple aggregate like count. + * Counts the occurrence of numbers (note that normally people count words, this + * example counts numbers) + * @param countTopic Topic where numbers are stored + * @throws Exception + */ + public void count(String countTopic) throws Exception { + if (maybeSetupPhase(countTopic, "simple-benchmark-produce-count", false)) { + return; + } + + CountDownLatch latch = new CountDownLatch(1); + Properties props = setStreamProperties("simple-benchmark-count"); + final KafkaStreams streams = createCountStreams(props, countTopic, latch); + runGenericBenchmark(streams, "Streams Count Performance [records/latency/rec-sec/MB-sec counted]: ", latch); + } + /** * Measure the performance of a KStream-KTable left join. The setup is such that each * KStream record joins to exactly one element in the KTable */ public void kStreamKTableJoin(String kStreamTopic, String kTableTopic) throws Exception { - CountDownLatch latch = new CountDownLatch(numRecords); + if (maybeSetupPhase(kStreamTopic, "simple-benchmark-produce-kstream", false)) { + maybeSetupPhase(kTableTopic, "simple-benchmark-produce-ktable", false); + return; + } - // initialize topics - System.out.println("Initializing kStreamTopic " + kStreamTopic); - produce(kStreamTopic, VALUE_SIZE, "simple-benchmark-produce-kstream", numRecords, false, numRecords, false); - System.out.println("Initializing kTableTopic " + kTableTopic); - produce(kTableTopic, VALUE_SIZE, "simple-benchmark-produce-ktable", numRecords, true, numRecords, false); + CountDownLatch latch = new CountDownLatch(1); // setup join - Properties props = setJoinProperties("simple-benchmark-kstream-ktable-join"); + Properties props = setStreamProperties("simple-benchmark-kstream-ktable-join"); final KafkaStreams streams = createKafkaStreamsKStreamKTableJoin(props, kStreamTopic, kTableTopic, latch); // run benchmark - runJoinBenchmark(streams, "Streams KStreamKTable LeftJoin Performance [MB/s joined]: ", latch); + runGenericBenchmark(streams, "Streams KStreamKTable LeftJoin Performance [records/latency/rec-sec/MB-sec joined]: ", latch); } /** @@ -177,20 +315,19 @@ public void kStreamKTableJoin(String kStreamTopic, String kTableTopic) throws Ex * KStream record joins to exactly one element in the other KStream */ public void kStreamKStreamJoin(String kStreamTopic1, String kStreamTopic2) throws Exception { - CountDownLatch latch = new CountDownLatch(numRecords); + if (maybeSetupPhase(kStreamTopic1, "simple-benchmark-produce-kstream-topic1", false)) { + maybeSetupPhase(kStreamTopic2, "simple-benchmark-produce-kstream-topic2", false); + return; + } - // initialize topics - System.out.println("Initializing kStreamTopic " + kStreamTopic1); - produce(kStreamTopic1, VALUE_SIZE, "simple-benchmark-produce-kstream-topic1", numRecords, true, numRecords, false); - System.out.println("Initializing kStreamTopic " + kStreamTopic2); - produce(kStreamTopic2, VALUE_SIZE, "simple-benchmark-produce-kstream-topic2", numRecords, true, numRecords, false); + CountDownLatch latch = new CountDownLatch(1); // setup join - Properties props = setJoinProperties("simple-benchmark-kstream-kstream-join"); + Properties props = setStreamProperties("simple-benchmark-kstream-kstream-join"); final KafkaStreams streams = createKafkaStreamsKStreamKStreamJoin(props, kStreamTopic1, kStreamTopic2, latch); // run benchmark - runJoinBenchmark(streams, "Streams KStreamKStream LeftJoin Performance [MB/s joined]: ", latch); + runGenericBenchmark(streams, "Streams KStreamKStream LeftJoin Performance [records/latency/rec-sec/MB-sec joined]: ", latch); } /** @@ -198,23 +335,29 @@ public void kStreamKStreamJoin(String kStreamTopic1, String kStreamTopic2) throw * KTable record joins to exactly one element in the other KTable */ public void kTableKTableJoin(String kTableTopic1, String kTableTopic2) throws Exception { - CountDownLatch latch = new CountDownLatch(numRecords); - - // initialize topics - System.out.println("Initializing kTableTopic " + kTableTopic1); - produce(kTableTopic1, VALUE_SIZE, "simple-benchmark-produce-ktable-topic1", numRecords, true, numRecords, false); - System.out.println("Initializing kTableTopic " + kTableTopic2); - produce(kTableTopic2, VALUE_SIZE, "simple-benchmark-produce-ktable-topic2", numRecords, true, numRecords, false); + if (maybeSetupPhase(kTableTopic1, "simple-benchmark-produce-ktable-topic1", false)) { + maybeSetupPhase(kTableTopic2, "simple-benchmark-produce-ktable-topic2", false); + return; + } + CountDownLatch latch = new CountDownLatch(1); // setup join - Properties props = setJoinProperties("simple-benchmark-ktable-ktable-join"); + Properties props = setStreamProperties("simple-benchmark-ktable-ktable-join"); final KafkaStreams streams = createKafkaStreamsKTableKTableJoin(props, kTableTopic1, kTableTopic2, latch); // run benchmark - runJoinBenchmark(streams, "Streams KTableKTable LeftJoin Performance [MB/s joined]: ", latch); + runGenericBenchmark(streams, "Streams KTableKTable LeftJoin Performance [records/latency/rec-sec/MB-sec joined]: ", latch); } - private void runJoinBenchmark(final KafkaStreams streams, final String nameOfBenchmark, final CountDownLatch latch) { + private void printResults(final String nameOfBenchmark, final long latency) { + System.out.println(nameOfBenchmark + + processedRecords + "/" + + latency + "/" + + recordsPerSec(latency, processedRecords) + "/" + + megabytesPerSec(latency, processedBytes)); + } + + private void runGenericBenchmark(final KafkaStreams streams, final String nameOfBenchmark, final CountDownLatch latch) { streams.start(); long startTime = System.currentTimeMillis(); @@ -227,20 +370,12 @@ private void runJoinBenchmark(final KafkaStreams streams, final String nameOfBen } } long endTime = System.currentTimeMillis(); - - - System.out.println(nameOfBenchmark + megaBytePerSec(endTime - startTime, numRecords, KEY_SIZE + VALUE_SIZE)); + printResults(nameOfBenchmark, endTime - startTime); streams.close(); } - - - public void processStream(String topic) { - CountDownLatch latch = new CountDownLatch(1); - - final KafkaStreams streams = createKafkaStreams(topic, stateDir, kafka, zookeeper, latch); - + private long startStreamsThread(final KafkaStreams streams, final CountDownLatch latch) throws Exception { Thread thread = new Thread() { public void run() { streams.start(); @@ -260,96 +395,72 @@ public void run() { long endTime = System.currentTimeMillis(); - System.out.println("Streams Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime)); - streams.close(); try { thread.join(); } catch (Exception ex) { // ignore } + + return endTime - startTime; } - public void processStreamWithSink(String topic) { - CountDownLatch latch = new CountDownLatch(1); + public void processStream(final String topic) throws Exception { + if (maybeSetupPhase(topic, "simple-benchmark-process-stream-load", true)) { + return; + } - final KafkaStreams streams = createKafkaStreamsWithSink(topic, stateDir, kafka, zookeeper, latch); + CountDownLatch latch = new CountDownLatch(1); - Thread thread = new Thread() { - public void run() { - streams.start(); - } - }; - thread.start(); + final KafkaStreams streams = createKafkaStreams(topic, latch); + long latency = startStreamsThread(streams, latch); - long startTime = System.currentTimeMillis(); + printResults("Streams Performance [records/latency/rec-sec/MB-sec source]: ", latency); + } - while (latch.getCount() > 0) { - try { - latch.await(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } + public void processStreamWithSink(String topic) throws Exception { + if (maybeSetupPhase(topic, "simple-benchmark-process-stream-with-sink-load", true)) { + return; } - long endTime = System.currentTimeMillis(); + CountDownLatch latch = new CountDownLatch(1); + final KafkaStreams streams = createKafkaStreamsWithSink(topic, latch); + long latency = startStreamsThread(streams, latch); - System.out.println("Streams Performance [MB/sec read+write]: " + megaBytePerSec(endTime - startTime)); + printResults("Streams Performance [records/latency/rec-sec/MB-sec source+sink]: ", latency); - streams.close(); - try { - thread.join(); - } catch (Exception ex) { - // ignore - } } - private void internalProcessStreamWithStore(final KafkaStreams streams, final CountDownLatch latch, - final String message) { - Thread thread = new Thread() { - public void run() { - streams.start(); - } - }; - thread.start(); - - long startTime = System.currentTimeMillis(); - - while (latch.getCount() > 0) { - try { - latch.await(); - } catch (InterruptedException ex) { - Thread.interrupted(); - } + public void processStreamWithStateStore(String topic) throws Exception { + if (maybeSetupPhase(topic, "simple-benchmark-process-stream-with-state-store-load", true)) { + return; } - long endTime = System.currentTimeMillis(); - - System.out.println(message + megaBytePerSec(endTime - startTime)); - - streams.close(); - try { - thread.join(); - } catch (Exception ex) { - // ignore - } - } - public void processStreamWithStateStore(String topic) { CountDownLatch latch = new CountDownLatch(1); - - final KafkaStreams streams = createKafkaStreamsWithStateStore(topic, stateDir, kafka, zookeeper, latch, false); - internalProcessStreamWithStore(streams, latch, "Streams Performance [MB/sec read+store]: "); + final KafkaStreams streams = createKafkaStreamsWithStateStore(topic, latch, false); + long latency = startStreamsThread(streams, latch); + printResults("Streams Performance [records/latency/rec-sec/MB-sec source+store]: ", latency); } - public void processStreamWithCachedStateStore(String topic) { - CountDownLatch latch = new CountDownLatch(1); - - final KafkaStreams streams = createKafkaStreamsWithStateStore(topic, stateDir, kafka, zookeeper, latch, true); + public void processStreamWithCachedStateStore(String topic) throws Exception { + if (maybeSetupPhase(topic, "simple-benchmark-process-stream-with-cached-state-store-load", true)) { + return; + } - internalProcessStreamWithStore(streams, latch, "Streams Performance [MB/sec read+cache+store]: "); + CountDownLatch latch = new CountDownLatch(1); + final KafkaStreams streams = createKafkaStreamsWithStateStore(topic, latch, true); + long latency = startStreamsThread(streams, latch); + printResults("Streams Performance [records/latency/rec-sec/MB-sec source+cache+store]: ", latency); } + public void produce(String topic) throws Exception { + // loading phase does not make sense for producer + if (loadPhase) { + return; + } + produce(topic, VALUE_SIZE, "simple-benchmark-produce", numRecords, true, numRecords, true); + } /** * Produce values to a topic * @param topic Topic to produce to @@ -361,22 +472,28 @@ public void processStreamWithCachedStateStore(String topic) { * @param printStats if True, print stats on how long producing took. If False, don't print stats. False can be used * when this produce step is part of another benchmark that produces its own stats */ - public void produce(String topic, int valueSizeBytes, String clientId, int numRecords, boolean sequential, - int upperRange, boolean printStats) throws Exception { + private void produce(String topic, int valueSizeBytes, String clientId, int numRecords, boolean sequential, + int upperRange, boolean printStats) throws Exception { + processedRecords = 0; + processedBytes = 0; if (sequential) { if (upperRange < numRecords) throw new Exception("UpperRange must be >= numRecords"); } - Properties props = new Properties(); - props.put(ProducerConfig.CLIENT_ID_CONFIG, clientId); - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); - props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + if (!sequential) { + System.out.println("WARNING: You are using non-sequential keys. If your tests' exit logic expects to see a final key, random keys may not work."); + } + Properties props = setProduceConsumeProperties(clientId); + int key = 0; Random rand = new Random(); KafkaProducer producer = new KafkaProducer<>(props); byte[] value = new byte[valueSizeBytes]; + // put some random values to increase entropy. Some devices + // like SSDs do compression and if the array is all zeros + // the performance will be too good. + new Random().nextBytes(value); long startTime = System.currentTimeMillis(); if (sequential) key = 0; @@ -385,22 +502,24 @@ public void produce(String topic, int valueSizeBytes, String clientId, int numRe producer.send(new ProducerRecord<>(topic, key, value)); if (sequential) key++; else key = rand.nextInt(upperRange); + processedRecords++; + processedBytes += value.length + Integer.SIZE; } producer.close(); long endTime = System.currentTimeMillis(); - if (printStats) - System.out.println("Producer Performance [MB/sec write]: " + megaBytePerSec(endTime - startTime, numRecords, KEY_SIZE + valueSizeBytes)); + if (printStats) { + printResults("Producer Performance [records/latency/rec-sec/MB-sec write]: ", endTime - startTime); + } } - public void consume(String topic) { - Properties props = new Properties(); - props.put(ConsumerConfig.CLIENT_ID_CONFIG, "simple-benchmark-consumer"); - props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); - props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + public void consume(String topic) throws Exception { + if (maybeSetupPhase(topic, "simple-benchmark-consumer-load", true)) { + return; + } + + Properties props = setProduceConsumeProperties("simple-benchmark-consumer"); KafkaConsumer consumer = new KafkaConsumer<>(props); @@ -413,33 +532,33 @@ public void consume(String topic) { long startTime = System.currentTimeMillis(); while (true) { - ConsumerRecords records = consumer.poll(500); + ConsumerRecords records = consumer.poll(POLL_MS); if (records.isEmpty()) { - if (endKey.equals(key)) + if (processedRecords == numRecords) break; } else { for (ConsumerRecord record : records) { + processedRecords++; + processedBytes += record.value().length + Integer.SIZE; Integer recKey = record.key(); - if (key == null || key < recKey) key = recKey; + if (processedRecords == numRecords) + break; } } + if (processedRecords == numRecords) + break; } long endTime = System.currentTimeMillis(); consumer.close(); - System.out.println("Consumer Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime)); + printResults("Consumer Performance [records/latency/rec-sec/MB-sec read]: ", endTime - startTime); } - private KafkaStreams createKafkaStreams(String topic, File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { - Properties props = new Properties(); - props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams"); - props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); - props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + private KafkaStreams createKafkaStreams(String topic, final CountDownLatch latch) { + Properties props = setStreamProperties("simple-benchmark-streams"); KStreamBuilder builder = new KStreamBuilder(); @@ -456,7 +575,9 @@ public void init(ProcessorContext context) { @Override public void process(Integer key, byte[] value) { - if (endKey.equals(key)) { + processedRecords++; + processedBytes += value.length + Integer.SIZE; + if (processedRecords == numRecords) { latch.countDown(); } } @@ -472,16 +593,11 @@ public void close() { } }); - return new KafkaStreams(builder, props); + return createKafkaStreamsWithExceptionHandler(builder, props); } - private KafkaStreams createKafkaStreamsWithSink(String topic, File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { - Properties props = new Properties(); - props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-sink"); - props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); - props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + private KafkaStreams createKafkaStreamsWithSink(String topic, final CountDownLatch latch) { + final Properties props = setStreamProperties("simple-benchmark-streams-with-sink"); KStreamBuilder builder = new KStreamBuilder(); @@ -498,7 +614,9 @@ public void init(ProcessorContext context) { @Override public void process(Integer key, byte[] value) { - if (endKey.equals(key)) { + processedRecords++; + processedBytes += value.length + Integer.SIZE; + if (processedRecords == numRecords) { latch.countDown(); } } @@ -514,17 +632,27 @@ public void close() { } }); - return new KafkaStreams(builder, props); + return createKafkaStreamsWithExceptionHandler(builder, props); } - private class CountDownAction implements ForeachAction { + private class CountDownAction implements ForeachAction { private CountDownLatch latch; CountDownAction(final CountDownLatch latch) { this.latch = latch; } @Override - public void apply(K key, V value) { - this.latch.countDown(); + public void apply(Integer key, V value) { + processedRecords++; + if (value instanceof byte[]) { + processedBytes += ((byte[]) value).length + Integer.SIZE; + } else if (value instanceof Long) { + processedBytes += Long.SIZE + Integer.SIZE; + } else { + System.err.println("Unknown value type in CountDownAction"); + } + if (processedRecords == numRecords) { + this.latch.countDown(); + } } } @@ -537,7 +665,7 @@ private KafkaStreams createKafkaStreamsKStreamKTableJoin(Properties streamConfig input1.leftJoin(input2, VALUE_JOINER).foreach(new CountDownAction(latch)); - return new KafkaStreams(builder, streamConfig); + return createKafkaStreamsWithExceptionHandler(builder, streamConfig); } private KafkaStreams createKafkaStreamsKTableKTableJoin(Properties streamConfig, String kTableTopic1, @@ -549,7 +677,7 @@ private KafkaStreams createKafkaStreamsKTableKTableJoin(Properties streamConfig, input1.leftJoin(input2, VALUE_JOINER).foreach(new CountDownAction(latch)); - return new KafkaStreams(builder, streamConfig); + return createKafkaStreamsWithExceptionHandler(builder, streamConfig); } private KafkaStreams createKafkaStreamsKStreamKStreamJoin(Properties streamConfig, String kStreamTopic1, @@ -562,18 +690,13 @@ private KafkaStreams createKafkaStreamsKStreamKStreamJoin(Properties streamConfi input1.leftJoin(input2, VALUE_JOINER, JoinWindows.of(timeDifferenceMs)).foreach(new CountDownAction(latch)); - return new KafkaStreams(builder, streamConfig); + return createKafkaStreamsWithExceptionHandler(builder, streamConfig); } - private KafkaStreams createKafkaStreamsWithStateStore(String topic, File stateDir, String kafka, String zookeeper, + private KafkaStreams createKafkaStreamsWithStateStore(String topic, final CountDownLatch latch, boolean enableCaching) { - Properties props = new Properties(); - props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-store" + enableCaching); - props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); - props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); - props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + Properties props = setStreamProperties("simple-benchmark-streams-with-store" + enableCaching); KStreamBuilder builder = new KStreamBuilder(); @@ -599,8 +722,9 @@ public void init(ProcessorContext context) { @Override public void process(Integer key, byte[] value) { store.put(key, value); - - if (endKey.equals(key)) { + processedRecords++; + processedBytes += value.length + Integer.SIZE; + if (processedRecords == numRecords) { latch.countDown(); } } @@ -616,15 +740,29 @@ public void close() { } }, "store"); - return new KafkaStreams(builder, props); + return createKafkaStreamsWithExceptionHandler(builder, props); } - private double megaBytePerSec(long time) { - return (double) (RECORD_SIZE * numRecords / 1024 / 1024) / ((double) time / 1000); + private KafkaStreams createKafkaStreamsWithExceptionHandler(final KStreamBuilder builder, final Properties props) { + final KafkaStreams streamsClient = new KafkaStreams(builder, props); + streamsClient.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + + streamsClient.close(30, TimeUnit.SECONDS); + } + }); + + return streamsClient; + } + + private double megabytesPerSec(long time, long processedBytes) { + return (processedBytes / 1024.0 / 1024.0) / (time / 1000.0); } - private double megaBytePerSec(long time, int numRecords, int recordSizeBytes) { - return (double) (recordSizeBytes * numRecords / 1024 / 1024) / ((double) time / 1000); + private double recordsPerSec(long time, int numRecords) { + return numRecords / (time / 1000.0); } private List getAllPartitions(KafkaConsumer consumer, String... topics) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java index e36bde4775876..e26453dc15846 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/DefaultPartitionGrouperTest.java @@ -21,8 +21,6 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; - -import static org.apache.kafka.common.utils.Utils.mkSet; import org.junit.Test; import java.util.Arrays; @@ -32,11 +30,12 @@ import java.util.Map; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.Assert.assertEquals; public class DefaultPartitionGrouperTest { - private List infos = Arrays.asList( + private final List infos = Arrays.asList( new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]), new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0]), @@ -44,39 +43,64 @@ public class DefaultPartitionGrouperTest { new PartitionInfo("topic2", 1, Node.noNode(), new Node[0], new Node[0]) ); - private Cluster metadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), infos, Collections.emptySet(), + private final Cluster metadata = new Cluster( + "cluster", + Collections.singletonList(Node.noNode()), + infos, + Collections.emptySet(), Collections.emptySet()); @Test - public void testGrouping() { - PartitionGrouper grouper = new DefaultPartitionGrouper(); - int topicGroupId; - Map> expectedPartitionsForTask; - Map> topicGroups; - - topicGroups = new HashMap<>(); - topicGroups.put(0, mkSet("topic1")); - topicGroups.put(1, mkSet("topic2")); - - expectedPartitionsForTask = new HashMap<>(); - topicGroupId = 0; + public void shouldComputeGroupingForTwoGroups() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); + + int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1")); expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic1", 0))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic1", 1))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 2), mkSet(new TopicPartition("topic1", 2))); - topicGroupId++; + + topicGroups.put(++topicGroupId, mkSet("topic2")); expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic2", 0))); expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic2", 1))); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); + } - topicGroups = new HashMap<>(); - topicGroups.put(0, mkSet("topic1", "topic2")); + @Test + public void shouldComputeGroupingForSingleGroupWithMultipleTopics() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); - expectedPartitionsForTask = new HashMap<>(); - topicGroupId = 0; - expectedPartitionsForTask.put(new TaskId(topicGroupId, 0), mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 1), mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); - expectedPartitionsForTask.put(new TaskId(topicGroupId, 2), mkSet(new TopicPartition("topic1", 2))); + final int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1", "topic2")); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 0), + mkSet(new TopicPartition("topic1", 0), new TopicPartition("topic2", 0))); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 1), + mkSet(new TopicPartition("topic1", 1), new TopicPartition("topic2", 1))); + expectedPartitionsForTask.put( + new TaskId(topicGroupId, 2), + mkSet(new TopicPartition("topic1", 2))); + + assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); + } + + @Test + public void shouldNotCreateAnyTasksBecauseOneTopicHasUnknownPartitions() { + final PartitionGrouper grouper = new DefaultPartitionGrouper(); + final Map> expectedPartitionsForTask = new HashMap<>(); + final Map> topicGroups = new HashMap<>(); + + final int topicGroupId = 0; + + topicGroups.put(topicGroupId, mkSet("topic1", "unknownTopic", "topic2")); assertEquals(expectedPartitionsForTask, grouper.partitionGroups(topicGroups, metadata)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index 4712320d160ed..a4d94a190388f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -50,6 +50,7 @@ import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; public class TopologyBuilderTest { @@ -156,12 +157,9 @@ public void testSourceTopics() { builder.addSource("source-3", "topic-3"); builder.addInternalTopic("topic-3"); - Set expected = new HashSet<>(); - expected.add("topic-1"); - expected.add("topic-2"); - expected.add("X-topic-3"); + Pattern expectedPattern = Pattern.compile("X-topic-3|topic-1|topic-2"); - assertEquals(expected, builder.sourceTopics()); + assertEquals(expectedPattern.pattern(), builder.sourceTopicPattern().pattern()); } @Test @@ -184,7 +182,7 @@ public void testAddMoreThanOnePatternSourceNode() { @Test public void testSubscribeTopicNameAndPattern() { final TopologyBuilder builder = new TopologyBuilder(); - Pattern expectedPattern = Pattern.compile("topic-foo|topic-bar|.*-\\d"); + Pattern expectedPattern = Pattern.compile("topic-bar|topic-foo|.*-\\d"); builder.addSource("source-1", "topic-foo", "topic-bar"); builder.addSource("source-2", Pattern.compile(".*-\\d")); assertEquals(expectedPattern.pattern(), builder.sourceTopicPattern().pattern()); @@ -441,9 +439,9 @@ public void shouldAssociateStateStoreNameWhenStateStoreSupplierIsInternal() thro builder.addSource("source", "topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("topic"), stateStoreNameToSourceTopic.get("store")); } @Test @@ -452,9 +450,9 @@ public void shouldAssociateStateStoreNameWhenStateStoreSupplierIsExternal() thro builder.addSource("source", "topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("topic"), stateStoreNameToSourceTopic.get("store")); } @Test @@ -465,9 +463,9 @@ public void shouldCorrectlyMapStateStoreToInternalTopics() throws Exception { builder.addSource("source", "internal-topic"); builder.addProcessor("processor", new MockProcessorSupplier(), "source"); builder.addStateStore(new MockStateStoreSupplier("store", false), "processor"); - final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); + final Map> stateStoreNameToSourceTopic = builder.stateStoreNameToSourceTopics(); assertEquals(1, stateStoreNameToSourceTopic.size()); - assertEquals(Collections.singleton("appId-internal-topic"), stateStoreNameToSourceTopic.get("store")); + assertEquals(Collections.singletonList("appId-internal-topic"), stateStoreNameToSourceTopic.get("store")); } @SuppressWarnings("unchecked") @@ -613,4 +611,37 @@ public void shouldSetCorrectSourceNodesWithRegexUpdatedTopics() throws Exception assertTrue(topicGroups.get(2).sourceTopics.contains("topic-3")); } + + @SuppressWarnings("unchecked") + @Test + public void shouldConnectRegexMatchedTopicsToStateStore() throws Exception { + + final TopologyBuilder topologyBuilder = new TopologyBuilder() + .addSource("ingest", Pattern.compile("topic-\\d+")) + .addProcessor("my-processor", new MockProcessorSupplier(), "ingest") + .addStateStore(new MockStateStoreSupplier("testStateStore", false), "my-processor"); + + final StreamPartitionAssignor.SubscriptionUpdates subscriptionUpdates = new StreamPartitionAssignor.SubscriptionUpdates(); + final Field updatedTopicsField = subscriptionUpdates.getClass().getDeclaredField("updatedTopicSubscriptions"); + updatedTopicsField.setAccessible(true); + + final Set updatedTopics = (Set) updatedTopicsField.get(subscriptionUpdates); + + updatedTopics.add("topic-2"); + updatedTopics.add("topic-3"); + updatedTopics.add("topic-A"); + + topologyBuilder.updateSubscriptions(subscriptionUpdates, "test-thread"); + topologyBuilder.setApplicationId("test-app"); + + Map> stateStoreAndTopics = topologyBuilder.stateStoreNameToSourceTopics(); + List topics = stateStoreAndTopics.get("testStateStore"); + + assertTrue("Expected to contain two topics", topics.size() == 2); + + assertTrue(topics.contains("topic-2")); + assertTrue(topics.contains("topic-3")); + assertFalse(topics.contains("topic-A")); + } + } 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 16967bcd1b0a4..a2346fc4cfa1b 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 @@ -71,6 +71,7 @@ private AbstractTask createTask(final Consumer consumer) { consumer, consumer, false, + new StateDirectory("app", TestUtils.tempDirectory().getPath()), new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))) { @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java new file mode 100644 index 0000000000000..e7235b03c9bc3 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/CopartitionedTopicsValidatorTest.java @@ -0,0 +1,137 @@ +/** + * 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 org.apache.kafka.common.Cluster; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.errors.TopologyBuilderException; +import org.junit.Before; +import org.junit.Test; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; + +public class CopartitionedTopicsValidatorTest { + + private final StreamPartitionAssignor.CopartitionedTopicsValidator validator + = new StreamPartitionAssignor.CopartitionedTopicsValidator("thread"); + private final Map partitions = new HashMap<>(); + private final Cluster cluster = Cluster.empty(); + + @Before + public void before() { + partitions.put(new TopicPartition("first", 0), new PartitionInfo("first", 0, null, null, null)); + partitions.put(new TopicPartition("first", 1), new PartitionInfo("first", 1, null, null, null)); + partitions.put(new TopicPartition("second", 0), new PartitionInfo("second", 0, null, null, null)); + partitions.put(new TopicPartition("second", 1), new PartitionInfo("second", 1, null, null, null)); + } + + @Test(expected = TopologyBuilderException.class) + public void shouldThrowTopologyBuilderExceptionIfNoPartitionsFoundForCoPartitionedTopic() throws Exception { + validator.validate(Collections.singleton("topic"), + Collections.emptyMap(), + cluster); + } + + @Test(expected = TopologyBuilderException.class) + public void shouldThrowTopologyBuilderExceptionIfPartitionCountsForCoPartitionedTopicsDontMatch() throws Exception { + partitions.remove(new TopicPartition("second", 0)); + validator.validate(Utils.mkSet("first", "second"), + Collections.emptyMap(), + cluster.withPartitions(partitions)); + } + + + @Test + public void shouldEnforceCopartitioningOnRepartitionTopics() throws Exception { + final StreamPartitionAssignor.InternalTopicMetadata metadata = createTopicMetadata("repartitioned", 10); + + validator.validate(Utils.mkSet("first", "second", metadata.config.name()), + Collections.singletonMap(metadata.config.name(), + metadata), + cluster.withPartitions(partitions)); + + assertThat(metadata.numPartitions, equalTo(2)); + } + + + @Test + public void shouldSetNumPartitionsToMaximumPartitionsWhenAllTopicsAreRepartitionTopics() throws Exception { + final StreamPartitionAssignor.InternalTopicMetadata one = createTopicMetadata("one", 1); + final StreamPartitionAssignor.InternalTopicMetadata two = createTopicMetadata("two", 15); + final StreamPartitionAssignor.InternalTopicMetadata three = createTopicMetadata("three", 5); + final Map repartitionTopicConfig = new HashMap<>(); + + repartitionTopicConfig.put(one.config.name(), one); + repartitionTopicConfig.put(two.config.name(), two); + repartitionTopicConfig.put(three.config.name(), three); + + validator.validate(Utils.mkSet(one.config.name(), + two.config.name(), + three.config.name()), + repartitionTopicConfig, + cluster + ); + + assertThat(one.numPartitions, equalTo(15)); + assertThat(two.numPartitions, equalTo(15)); + assertThat(three.numPartitions, equalTo(15)); + } + + @Test + public void shouldSetRepartitionTopicsPartitionCountToNotAvailableIfAnyNotAvaliable() throws Exception { + final StreamPartitionAssignor.InternalTopicMetadata one = createTopicMetadata("one", 1); + final StreamPartitionAssignor.InternalTopicMetadata two = createTopicMetadata("two", StreamPartitionAssignor.NOT_AVAILABLE); + final Map repartitionTopicConfig = new HashMap<>(); + + repartitionTopicConfig.put(one.config.name(), one); + repartitionTopicConfig.put(two.config.name(), two); + + validator.validate(Utils.mkSet("first", + "second", + one.config.name(), + two.config.name()), + repartitionTopicConfig, + cluster.withPartitions(partitions)); + + assertThat(one.numPartitions, equalTo(StreamPartitionAssignor.NOT_AVAILABLE)); + assertThat(two.numPartitions, equalTo(StreamPartitionAssignor.NOT_AVAILABLE)); + + } + + private StreamPartitionAssignor.InternalTopicMetadata createTopicMetadata(final String repartitionTopic, + final int partitions) { + final InternalTopicConfig repartitionTopicConfig + = new InternalTopicConfig(repartitionTopic, + Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), + Collections.emptyMap()); + + + final StreamPartitionAssignor.InternalTopicMetadata metadata + = new StreamPartitionAssignor.InternalTopicMetadata(repartitionTopicConfig); + metadata.numPartitions = partitions; + return metadata; + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java index db51cefabf587..8c9cf19cb93ba 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateManagerImplTest.java @@ -49,6 +49,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -122,15 +124,15 @@ public void shouldReadCheckpointOffsets() throws Exception { final Map expected = writeCheckpoint(); stateManager.initialize(context); - final Map offsets = stateManager.checkpointedOffsets(); + final Map offsets = stateManager.checkpointed(); assertEquals(expected, offsets); } @Test - public void shouldDeleteCheckpointFileAfteLoaded() throws Exception { + public void shouldNotDeleteCheckpointFileAfterLoaded() throws Exception { writeCheckpoint(); stateManager.initialize(context); - assertFalse(checkpointFile.exists()); + assertTrue(checkpointFile.exists()); } @Test(expected = StreamsException.class) @@ -168,7 +170,7 @@ public void shouldThrowIllegalArgumentIfTryingToRegisterStoreThatIsNotGlobal() t } @Test - public void shouldThrowIllegalArgumenExceptionIfAttemptingToRegisterStoreTwice() throws Exception { + public void shouldThrowIllegalArgumentExceptionIfAttemptingToRegisterStoreTwice() throws Exception { stateManager.initialize(context); initializeConsumer(2, 1, t1); stateManager.register(store1, false, new TheStateRestoreCallback()); @@ -271,9 +273,7 @@ public void shouldWriteCheckpointsOnClose() throws Exception { stateManager.register(store1, false, stateRestoreCallback); final Map expected = Collections.singletonMap(t1, 25L); stateManager.close(expected); - final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint(new File(stateManager.baseDir(), - ProcessorStateManager.CHECKPOINT_FILE_NAME)); - final Map result = offsetCheckpoint.read(); + final Map result = readOffsetsCheckpoint(); assertEquals(expected, result); } @@ -379,6 +379,41 @@ public void shouldReleaseLockIfExceptionWhenLoadingCheckpoints() throws Exceptio } } + @Test + public void shouldCheckpointOffsets() throws Exception { + final Map offsets = Collections.singletonMap(t1, 25L); + stateManager.initialize(context); + + stateManager.checkpoint(offsets); + + final Map result = readOffsetsCheckpoint(); + assertThat(result, equalTo(offsets)); + assertThat(stateManager.checkpointed(), equalTo(offsets)); + } + + @Test + public void shouldNotRemoveOffsetsOfUnUpdatedTablesDuringCheckpoint() throws Exception { + stateManager.initialize(context); + final TheStateRestoreCallback stateRestoreCallback = new TheStateRestoreCallback(); + initializeConsumer(10, 1, t1); + stateManager.register(store1, false, stateRestoreCallback); + initializeConsumer(20, 1, t2); + stateManager.register(store2, false, stateRestoreCallback); + + final Map initialCheckpoint = stateManager.checkpointed(); + stateManager.checkpoint(Collections.singletonMap(t1, 101L)); + + final Map updatedCheckpoint = stateManager.checkpointed(); + assertThat(updatedCheckpoint.get(t2), equalTo(initialCheckpoint.get(t2))); + assertThat(updatedCheckpoint.get(t1), equalTo(101L)); + } + + private Map readOffsetsCheckpoint() throws IOException { + final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint(new File(stateManager.baseDir(), + ProcessorStateManager.CHECKPOINT_FILE_NAME)); + return offsetCheckpoint.read(); + } + private void initializeConsumer(final long numRecords, final long startOffset, final TopicPartition topicPartition) { final HashMap startOffsets = new HashMap<>(); startOffsets.put(topicPartition, 1L); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java index df0b73ca8a4cd..66999bc8efbe1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/GlobalStateTaskTest.java @@ -38,6 +38,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -137,7 +139,19 @@ public void shouldCloseStateManagerWithOffsets() throws Exception { globalStateTask.initialize(); globalStateTask.update(new ConsumerRecord<>("t1", 1, 51, "foo".getBytes(), "foo".getBytes())); globalStateTask.close(); - assertEquals(expectedOffsets, stateMgr.checkpointedOffsets()); + assertEquals(expectedOffsets, stateMgr.checkpointed()); assertTrue(stateMgr.closed); } + + @Test + public void shouldCheckpointOffsetsWhenStateIsFlushed() throws Exception { + final Map expectedOffsets = new HashMap<>(); + expectedOffsets.put(t1, 102L); + expectedOffsets.put(t2, 100L); + globalStateTask.initialize(); + globalStateTask.update(new ConsumerRecord<>("t1", 1, 101, "foo".getBytes(), "foo".getBytes())); + globalStateTask.flushState(); + assertThat(stateMgr.checkpointed(), equalTo(expectedOffsets)); + } + } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java new file mode 100644 index 0000000000000..2169496b95607 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/InternalTopicManagerTest.java @@ -0,0 +1,118 @@ +/** + * 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 org.apache.kafka.common.Node; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.requests.MetadataResponse; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.errors.StreamsException; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; + +import static org.apache.kafka.streams.processor.internals.InternalTopicManager.WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT; + +public class InternalTopicManagerTest { + + private final String topic = "test_topic"; + private final String userEndPoint = "localhost:2171"; + private MockStreamKafkaClient streamsKafkaClient; + private final Time time = new MockTime(); + @Before + public void init() { + final StreamsConfig config = new StreamsConfig(configProps()); + streamsKafkaClient = new MockStreamKafkaClient(config); + } + + @After + public void shutdown() throws IOException { + streamsKafkaClient.close(); + } + + @Test + public void shouldReturnCorrectPartitionCounts() throws Exception { + InternalTopicManager internalTopicManager = new InternalTopicManager(streamsKafkaClient, 1, + WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT, time); + Assert.assertEquals(Collections.singletonMap(topic, 1), internalTopicManager.getNumPartitions(Collections.singleton(topic))); + } + + @Test + public void shouldCreateRequiredTopics() throws Exception { + InternalTopicManager internalTopicManager = new InternalTopicManager(streamsKafkaClient, 1, + WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT, time); + internalTopicManager.makeReady(Collections.singletonMap(new InternalTopicConfig(topic, Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), null), 1)); + } + + @Test + public void shouldNotCreateTopicIfExistsWithDifferentPartitions() throws Exception { + InternalTopicManager internalTopicManager = new InternalTopicManager(streamsKafkaClient, 1, + WINDOW_CHANGE_LOG_ADDITIONAL_RETENTION_DEFAULT, time); + boolean exceptionWasThrown = false; + try { + internalTopicManager.makeReady(Collections.singletonMap(new InternalTopicConfig(topic, Collections.singleton(InternalTopicConfig.CleanupPolicy.compact), null), 2)); + } catch (StreamsException e) { + exceptionWasThrown = true; + } + Assert.assertTrue(exceptionWasThrown); + } + + private Properties configProps() { + return new Properties() { + { + setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "Internal-Topic-ManagerTest"); + setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, userEndPoint); + setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); + } + }; + } + + private class MockStreamKafkaClient extends StreamsKafkaClient { + + MockStreamKafkaClient(final StreamsConfig streamsConfig) { + super(streamsConfig); + } + + @Override + public void createTopics(final Map topicsMap, final int replicationFactor, + final long windowChangeLogAdditionalRetention, final MetadataResponse metadata) { + // do nothing + } + + @Override + public MetadataResponse fetchMetadata() { + Node node = new Node(1, "host1", 1001); + MetadataResponse.PartitionMetadata partitionMetadata = new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, new ArrayList(), new ArrayList()); + MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(Errors.NONE, topic, true, Collections.singletonList(partitionMetadata)); + MetadataResponse response = new MetadataResponse(Collections.singletonList(node), null, MetadataResponse.NO_CONTROLLER_ID, + Collections.singletonList(topicMetadata)); + return response; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java index 0a1f95c29e0a8..52435197240eb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java @@ -1,10 +1,10 @@ /** * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with + * 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 + * the License. You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * @@ -14,80 +14,65 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.kafka.streams.processor.internals; -import static org.junit.Assert.assertEquals; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import org.junit.Test; public class MinTimestampTrackerTest { - private Stamped elem(long timestamp) { - return new Stamped<>("", timestamp); - } + private MinTimestampTracker tracker = new MinTimestampTracker<>(); - @SuppressWarnings("unchecked") @Test - public void testTracking() { - TimestampTracker tracker = new MinTimestampTracker<>(); - - Object[] elems = new Object[]{ - elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) - }; - - int insertionIndex = 0; - int removalIndex = 0; - - // add 100 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // add 101 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(100L, tracker.get()); - - // remove 100 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(101L, tracker.get()); - - // add 102 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(101L, tracker.get()); - - // add 98 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 99 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); - - // add 100 - tracker.addElement((Stamped) elems[insertionIndex++]); - assertEquals(98L, tracker.get()); + public void shouldReturnNotKnownTimestampWhenNoRecordsEverAdded() throws Exception { + assertThat(tracker.get(), equalTo(TimestampTracker.NOT_KNOWN)); + } - // remove 101 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); + @Test + public void shouldReturnTimestampOfOnlyRecord() throws Exception { + tracker.addElement(elem(100)); + assertThat(tracker.get(), equalTo(100L)); + } - // remove 102 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(98L, tracker.get()); + @Test + public void shouldReturnLowestAvailableTimestampFromAllInputs() throws Exception { + tracker.addElement(elem(100)); + tracker.addElement(elem(99)); + tracker.addElement(elem(102)); + assertThat(tracker.get(), equalTo(99L)); + } - // remove 98 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(99L, tracker.get()); + @Test + public void shouldReturnLowestAvailableTimestampAfterPreviousLowestRemoved() throws Exception { + final Stamped lowest = elem(88); + tracker.addElement(lowest); + tracker.addElement(elem(101)); + tracker.addElement(elem(99)); + tracker.removeElement(lowest); + assertThat(tracker.get(), equalTo(99L)); + } - // remove 99 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(100L, tracker.get()); + @Test + public void shouldReturnLastKnownTimestampWhenAllElementsHaveBeenRemoved() throws Exception { + final Stamped record = elem(98); + tracker.addElement(record); + tracker.removeElement(record); + assertThat(tracker.get(), equalTo(98L)); + } - // remove 100 - tracker.removeElement((Stamped) elems[removalIndex++]); - assertEquals(100L, tracker.get()); + @Test + public void shouldIgnoreNullRecordOnRemove() throws Exception { + tracker.removeElement(null); + } - assertEquals(insertionIndex, removalIndex); + @Test(expected = NullPointerException.class) + public void shouldThrowNullPointerExceptionWhenTryingToAddNullElement() throws Exception { + tracker.addElement(null); } + private Stamped elem(final long timestamp) { + return new Stamped<>("", timestamp); + } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java index b1d5f087dea1c..f2c7e53a0aa1c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorNodeTest.java @@ -103,7 +103,7 @@ public void testMetrics() { String name = "task." + context.taskId() + "." + node.name(); String[] entities = {"all", name}; String[] latencyOperations = {"process", "punctuate", "create", "destroy"}; - String throughputOperation = "process-throughput"; + String throughputOperation = "forward"; String groupName = "stream-processor-node-metrics"; Map tags = Collections.singletonMap("processor-node-id", node.name()); @@ -115,14 +115,14 @@ public void testMetrics() { for (String entity : entities) { for (String operation : latencyOperations) { - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-avg-latency", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-latency-avg", groupName, "The average latency in milliseconds of " + entity + " " + operation + " operation.", tags))); - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-max-latency", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-latency-max", groupName, "The max latency in milliseconds of " + entity + " " + operation + " operation.", tags))); - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-qps", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-rate", groupName, "The average number of occurrence of " + entity + " " + operation + " operation per second.", tags))); } - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + throughputOperation + "-qps", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + throughputOperation + "-rate", groupName, "The average number of occurrence of " + entity + " " + throughputOperation + " operation per second.", tags))); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index de547230ae3f9..a9998e6e2e189 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.errors.StreamsException; @@ -49,6 +50,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -57,14 +60,10 @@ public class ProcessorStateManagerTest { - private File baseDir; - private StateDirectory stateDirectory; - public static class MockRestoreConsumer extends MockConsumer { private final Serializer serializer = new IntegerSerializer(); private TopicPartition assignedPartition = null; - private TopicPartition seekPartition = null; private long seekOffset = -1L; private boolean seekToBeginingCalled = false; private boolean seekToEndCalled = false; @@ -155,7 +154,6 @@ public synchronized void seek(TopicPartition partition, long offset) { if (seekOffset >= 0) throw new IllegalStateException("RestoreConsumer: offset already seeked"); - seekPartition = partition; seekOffset = offset; currentOffset = offset; super.seek(partition, offset); @@ -196,11 +194,32 @@ public synchronized void seekToEnd(Collection partitions) { private final String nonPersistentStoreName = "nonPersistentStore"; private final String persistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, persistentStoreName); private final String nonPersistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, nonPersistentStoreName); + private final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); + private final MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); + private final TopicPartition persistentStorePartition = new TopicPartition(persistentStoreTopicName, 1); + private final String storeName = "mockStateStore"; + private final String changelogTopic = ProcessorStateManager.storeChangelogTopic(applicationId, storeName); + private final TopicPartition changelogTopicPartition = new TopicPartition(changelogTopic, 0); + private final TaskId taskId = new TaskId(0, 1); + private final MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + private final MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(storeName, true); + private File baseDir; + private File checkpointFile; + private StateDirectory stateDirectory; + private OffsetCheckpoint checkpoint; @Before public void setup() { baseDir = TestUtils.tempDirectory(); stateDirectory = new StateDirectory(applicationId, baseDir.getPath()); + checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); + checkpoint = new OffsetCheckpoint(checkpointFile); + restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( + new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( + new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); } @After @@ -212,7 +231,7 @@ public void cleanup() { public void testNoTopic() throws IOException { MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, new HashMap() { { put(nonPersistentStoreName, nonPersistentStoreName); } @@ -244,7 +263,7 @@ public void testRegisterPersistentStore() throws IOException { MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore("persistentStore", true); // persistent store - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreName); @@ -283,8 +302,6 @@ public void testRegisterPersistentStore() throws IOException { public void testRegisterNonPersistentStore() throws IOException { long lastCheckpointedOffset = 10L; - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); checkpoint.write(Collections.singletonMap(new TopicPartition(persistentStoreTopicName, 2), lastCheckpointedOffset)); @@ -296,9 +313,7 @@ public void testRegisterNonPersistentStore() throws IOException { TopicPartition partition = new TopicPartition(persistentStoreTopicName, 2); restoreConsumer.updateEndOffsets(Collections.singletonMap(partition, 13L)); - MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); // non persistent store - - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 2), noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); @@ -308,7 +323,7 @@ public void testRegisterNonPersistentStore() throws IOException { restoreConsumer.reset(); ArrayList expectedKeys = new ArrayList<>(); - long offset = -1L; + long offset; for (int i = 1; i <= 3; i++) { offset = (long) (i + 100); int key = i; @@ -329,12 +344,13 @@ public void testRegisterNonPersistentStore() throws IOException { } finally { stateMgr.close(Collections.emptyMap()); } - } @Test public void testChangeLogOffsets() throws IOException { final TaskId taskId = new TaskId(0, 0); + final OffsetCheckpoint offsetCheckpoint = new OffsetCheckpoint( + new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); long lastCheckpointedOffset = 10L; String storeName1 = "store1"; String storeName2 = "store2"; @@ -349,10 +365,7 @@ public void testChangeLogOffsets() throws IOException { storeToChangelogTopic.put(storeName2, storeTopicName2); storeToChangelogTopic.put(storeName3, storeTopicName3); - OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME)); - checkpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); - - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + offsetCheckpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset)); restoreConsumer.updatePartitions(storeTopicName1, Utils.mkList( new PartitionInfo(storeTopicName1, 0, Node.noNode(), new Node[0], new Node[0]) @@ -381,7 +394,7 @@ public void testChangeLogOffsets() throws IOException { // if there is an source partition, inherit the partition id Set sourcePartitions = Utils.mkSet(new TopicPartition(storeTopicName3, 1)); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, sourcePartitions, restoreConsumer, true, stateDirectory, storeToChangelogTopic); // standby try { restoreConsumer.reset(); @@ -389,7 +402,7 @@ public void testChangeLogOffsets() throws IOException { stateMgr.register(store2, true, store2.stateRestoreCallback); stateMgr.register(store3, true, store3.stateRestoreCallback); - Map changeLogOffsets = stateMgr.checkpointedOffsets(); + Map changeLogOffsets = stateMgr.checkpointed(); assertEquals(3, changeLogOffsets.size()); assertTrue(changeLogOffsets.containsKey(partition1)); @@ -407,20 +420,12 @@ public void testChangeLogOffsets() throws IOException { @Test public void testGetStore() throws IOException { - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); - - MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + final ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); try { - stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback); + stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); assertNull(stateMgr.getStore("noSuchStore")); - assertEquals(mockStateStore, stateMgr.getStore(nonPersistentStoreName)); + assertEquals(nonPersistentStore, stateMgr.getStore(nonPersistentStoreName)); } finally { stateMgr.close(Collections.emptyMap()); @@ -429,39 +434,24 @@ public void testGetStore() throws IOException { @Test public void testFlushAndClose() throws IOException { - final TaskId taskId = new TaskId(0, 1); - File checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); // write an empty checkpoint file - OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); - oldCheckpoint.write(Collections.emptyMap()); - - MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); - - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); - restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( - new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); + checkpoint.write(Collections.emptyMap()); // set up ack'ed offsets - HashMap ackedOffsets = new HashMap<>(); + final HashMap ackedOffsets = new HashMap<>(); ackedOffsets.put(new TopicPartition(persistentStoreTopicName, 1), 123L); ackedOffsets.put(new TopicPartition(nonPersistentStoreTopicName, 1), 456L); ackedOffsets.put(new TopicPartition(ProcessorStateManager.storeChangelogTopic(applicationId, "otherTopic"), 1), 789L); - MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); - MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { + ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, noPartitions, restoreConsumer, false, stateDirectory, new HashMap() { { put(persistentStoreName, persistentStoreTopicName); put(nonPersistentStoreName, nonPersistentStoreTopicName); } }); try { - // make sure the checkpoint file is deleted - assertFalse(checkpointFile.exists()); + // make sure the checkpoint file isn't deleted + assertTrue(checkpointFile.exists()); restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); @@ -482,42 +472,122 @@ public void testFlushAndClose() throws IOException { assertTrue(checkpointFile.exists()); // the checkpoint file should contain an offset from the persistent store only. - OffsetCheckpoint newCheckpoint = new OffsetCheckpoint(checkpointFile); - Map checkpointedOffsets = newCheckpoint.read(); + final Map checkpointedOffsets = checkpoint.read(); assertEquals(1, checkpointedOffsets.size()); assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition(persistentStoreTopicName, 1))); } @Test public void shouldRegisterStoreWithoutLoggingEnabledAndNotBackedByATopic() throws Exception { - MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); - ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); - stateMgr.register(mockStateStore, false, mockStateStore.stateRestoreCallback); + ProcessorStateManager stateMgr = new ProcessorStateManager(new TaskId(0, 1), noPartitions, new MockRestoreConsumer(), false, stateDirectory, Collections.emptyMap()); + stateMgr.register(nonPersistentStore, false, nonPersistentStore.stateRestoreCallback); assertNotNull(stateMgr.getStore(nonPersistentStoreName)); } + @Test - public void shouldNotWriteCheckpointsIfAckeOffsetsIsNull() throws Exception { - final TaskId taskId = new TaskId(0, 1); - final File checkpointFile = new File(stateDirectory.directoryForTask(taskId), ProcessorStateManager.CHECKPOINT_FILE_NAME); - // write an empty checkpoint file - final OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); - oldCheckpoint.write(Collections.emptyMap()); + public void shouldNotChangeOffsetsIfAckedOffsetsIsNull() throws Exception { + final Map offsets = Collections.singletonMap(persistentStorePartition, 99L); + checkpoint.write(offsets); - final MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + false, + stateDirectory, + Collections.emptyMap()); - restoreConsumer.updatePartitions(persistentStoreTopicName, Utils.mkList( - new PartitionInfo(persistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) - )); + restoreConsumer.reset(); + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); + stateMgr.close(null); + final Map read = checkpoint.read(); + assertThat(read, equalTo(offsets)); + } + @Test + public void shouldWriteCheckpointForPersistentLogEnabledStore() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + false, + stateDirectory, + Collections.singletonMap(persistentStore.name(), + persistentStoreTopicName)); + restoreConsumer.reset(); + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - final MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true); - final ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, taskId, noPartitions, restoreConsumer, false, stateDirectory, Collections.emptyMap()); + stateMgr.checkpoint(Collections.singletonMap(persistentStorePartition, 10L)); + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.singletonMap(persistentStorePartition, 11L))); + } + + @Test + public void shouldWriteCheckpointForStandbyReplica() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.singletonMap(persistentStore.name(), + persistentStoreTopicName)); restoreConsumer.reset(); stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); - stateMgr.close(null); - assertFalse(checkpointFile.exists()); + final byte[] bytes = Serdes.Integer().serializer().serialize("", 10); + stateMgr.updateStandbyStates(persistentStorePartition, + Collections.singletonList( + new ConsumerRecord<>(persistentStorePartition.topic(), + persistentStorePartition.partition(), + 888L, + bytes, + bytes))); + + stateMgr.checkpoint(Collections.emptyMap()); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.singletonMap(persistentStorePartition, 889L))); + + } + + @Test + public void shouldNotWriteCheckpointForNonPersistent() throws Exception { + final TopicPartition topicPartition = new TopicPartition(nonPersistentStoreTopicName, 1); + + restoreConsumer.updatePartitions(nonPersistentStoreTopicName, Utils.mkList( + new PartitionInfo(nonPersistentStoreTopicName, 1, Node.noNode(), new Node[0], new Node[0]) + )); + + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.singletonMap(nonPersistentStoreName, + nonPersistentStoreTopicName)); + + restoreConsumer.reset(); + stateMgr.register(nonPersistentStore, true, nonPersistentStore.stateRestoreCallback); + stateMgr.checkpoint(Collections.singletonMap(topicPartition, 876L)); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.emptyMap())); + } + + @Test + public void shouldNotWriteCheckpointForStoresWithoutChangelogTopic() throws Exception { + final ProcessorStateManager stateMgr = new ProcessorStateManager(taskId, + noPartitions, + restoreConsumer, + true, + stateDirectory, + Collections.emptyMap()); + + stateMgr.register(persistentStore, true, persistentStore.stateRestoreCallback); + + stateMgr.checkpoint(Collections.singletonMap(persistentStorePartition, 987L)); + + final Map read = checkpoint.read(); + assertThat(read, equalTo(Collections.emptyMap())); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java index b54ce34796484..f0d06f715d7de 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java @@ -74,14 +74,14 @@ public void testSpecificPartition() { new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer), "RecordCollectorTest-TestSpecificPartition"); - collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer); - collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer); - collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer); - collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer); - collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 1, null, stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 1, null, stringSerializer, stringSerializer); - collector.send(new ProducerRecord<>("topic1", 2, "999", "0"), stringSerializer, stringSerializer); + collector.send("topic1", "999", "0", 2, null, stringSerializer, stringSerializer); Map offsets = collector.offsets(); @@ -90,9 +90,9 @@ public void testSpecificPartition() { assertEquals((Long) 0L, offsets.get(new TopicPartition("topic1", 2))); // ignore StreamPartitioner - collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", 2, "999", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "999", "0", 0, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "999", "0", 1, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "999", "0", 2, null, stringSerializer, stringSerializer, streamPartitioner); assertEquals((Long) 3L, offsets.get(new TopicPartition("topic1", 0))); assertEquals((Long) 2L, offsets.get(new TopicPartition("topic1", 1))); @@ -106,17 +106,17 @@ public void testStreamPartitioner() { new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer), "RecordCollectorTest-TestStreamPartitioner"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "9", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "27", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "81", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "243", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "9", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "27", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "81", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "243", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "28", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "82", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "244", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "28", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "82", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "244", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "245", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "245", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); Map offsets = collector.offsets(); @@ -141,7 +141,7 @@ public synchronized Future send(final ProducerRecord record, fin }, "test"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); final Long offset = collector.offsets().get(new TopicPartition("topic1", 0)); assertEquals(Long.valueOf(0L), offset); } @@ -158,7 +158,7 @@ public synchronized Future send(final ProducerRecord record, fin }, "test"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); } @@ -174,8 +174,8 @@ public synchronized Future send(final ProducerRecord record, fin } }, "test"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); } @SuppressWarnings("unchecked") @@ -190,7 +190,7 @@ public synchronized Future send(final ProducerRecord record, fin } }, "test"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); collector.flush(); } @@ -206,7 +206,7 @@ public synchronized Future send(final ProducerRecord record, fin } }, "test"); - collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner); + collector.send("topic1", "3", "0", null, null, stringSerializer, stringSerializer, streamPartitioner); collector.close(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java index fdd9127cd2cb4..cf328eee808ca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/SourceNodeRecordDeserializerTest.java @@ -22,6 +22,8 @@ import org.apache.kafka.streams.errors.StreamsException; import org.junit.Test; +import java.util.Collections; + import static org.junit.Assert.assertEquals; public class SourceNodeRecordDeserializerTest { @@ -82,7 +84,7 @@ static class TheSourceNode extends SourceNode { final boolean valueThrowsException, final Object key, final Object value) { - super("", new String[0], null, null); + super("", Collections.EMPTY_LIST, null, null); this.keyThrowsException = keyThrowsException; this.valueThrowsException = valueThrowsException; this.key = key; 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 2d32e780faee3..4c3356a4f89e1 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 @@ -27,6 +27,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.StreamsConfig; @@ -53,6 +54,8 @@ import java.util.Set; import static java.util.Collections.singleton; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -315,7 +318,7 @@ public void shouldNotThrowUnsupportedOperationExceptionWhenInitializingStateStor final String changelogName = "test-application-my-store-changelog"; final List partitions = Utils.mkList(new TopicPartition(changelogName, 0)); consumer.assign(partitions); - Map committedOffsets = new HashMap<>(); + final Map committedOffsets = new HashMap<>(); committedOffsets.put(new TopicPartition(changelogName, 0), new OffsetAndMetadata(0L)); consumer.commitSync(committedOffsets); @@ -326,9 +329,51 @@ public void shouldNotThrowUnsupportedOperationExceptionWhenInitializingStateStor final ProcessorTopology topology = builder.setApplicationId(applicationId).build(0); StreamsConfig config = createConfig(baseDir); new StandbyTask(taskId, applicationId, partitions, topology, consumer, restoreStateConsumer, config, - new MockStreamsMetrics(new Metrics()), stateDirectory); + new MockStreamsMetrics(new Metrics()), stateDirectory); } + + @Test + public void shouldCheckpointStoreOffsetsOnCommit() throws Exception { + consumer.assign(Utils.mkList(ktable)); + final Map committedOffsets = new HashMap<>(); + committedOffsets.put(new TopicPartition(ktable.topic(), ktable.partition()), new OffsetAndMetadata(100L)); + consumer.commitSync(committedOffsets); + + restoreStateConsumer.updatePartitions("ktable1", Utils.mkList( + new PartitionInfo("ktable1", 0, Node.noNode(), new Node[0], new Node[0]))); + + final TaskId taskId = new TaskId(0, 0); + final StreamsConfig config = createConfig(baseDir); + final StandbyTask task = new StandbyTask(taskId, + applicationId, + ktablePartitions, + ktableTopology, + consumer, + restoreStateConsumer, + config, + null, + stateDirectory + ); + + + restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions())); + + final byte[] serializedValue = Serdes.Integer().serializer().serialize("", 1); + task.update(ktable, Collections.singletonList(new ConsumerRecord<>(ktable.topic(), + ktable.partition(), + 50L, + serializedValue, + serializedValue))); + + task.commit(); + + final Map checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), + ProcessorStateManager.CHECKPOINT_FILE_NAME)).read(); + assertThat(checkpoint, equalTo(Collections.singletonMap(ktable, 51L))); + + } + private List> records(ConsumerRecord... recs) { return Arrays.asList(recs); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java index fb55796a45558..6b1d0773928af 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StateDirectoryTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.test.TestUtils; import org.junit.After; @@ -32,6 +33,7 @@ import java.util.List; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; @@ -97,6 +99,22 @@ public void shouldBeTrueIfAlreadyHoldsLock() throws Exception { assertTrue(directory.lock(taskId, 0)); } + @Test(expected = ProcessorStateException.class) + public void shouldThrowProcessorStateException() throws Exception { + final TaskId taskId = new TaskId(0, 0); + + Utils.delete(stateDir); + directory.directoryForTask(taskId); + } + + @Test + public void shouldNotLockDeletedDirectory() throws Exception { + final TaskId taskId = new TaskId(0, 0); + + Utils.delete(stateDir); + assertFalse(directory.lock(taskId, 0)); + } + @Test public void shouldLockMulitpleTaskDirectories() throws Exception { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 5aa40c82733b5..6503038e5b3ef 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -17,9 +17,9 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -27,6 +27,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -39,23 +40,23 @@ import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.test.MockClientSupplier; +import org.apache.kafka.test.MockInternalTopicManager; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.MockTimestampExtractor; import org.apache.kafka.test.MockStateStoreSupplier; -import org.apache.kafka.test.MockInternalTopicManager; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Assert; 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.Set; -import java.util.Properties; import java.util.Map; +import java.util.Properties; +import java.util.Set; import java.util.UUID; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Collections; -import java.util.ArrayList; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; @@ -92,14 +93,18 @@ public class StreamPartitionAssignorTest { new PartitionInfo("topic3", 3, Node.noNode(), new Node[0], new Node[0]) ); - private Cluster metadata = new Cluster("cluster", Arrays.asList(Node.noNode()), infos, Collections.emptySet(), + private Cluster metadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), infos, Collections.emptySet(), Collections.emptySet()); private final TaskId task0 = new TaskId(0, 0); private final TaskId task1 = new TaskId(0, 1); private final TaskId task2 = new TaskId(0, 2); private final TaskId task3 = new TaskId(0, 3); - private String userEndPoint = "localhost:2171"; + private final String userEndPoint = "localhost:2171"; + private final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); + private final MockClientSupplier mockClientSupplier = new MockClientSupplier(); + private final TopologyBuilder builder = new TopologyBuilder(); + private final StreamsConfig config = new StreamsConfig(configProps()); private Properties configProps() { return new Properties() { @@ -115,9 +120,6 @@ private Properties configProps() { @SuppressWarnings("unchecked") @Test public void testSubscription() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); - - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); @@ -142,7 +144,6 @@ public Set cachedTasks() { } }; - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread, "test", clientId)); PartitionAssignor.Subscription subscription = partitionAssignor.subscription(Utils.mkSet("topic1", "topic2")); @@ -159,9 +160,6 @@ public Set cachedTasks() { @Test public void testAssignBasic() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); - - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); @@ -180,12 +178,10 @@ public void testAssignBasic() throws Exception { String client1 = "client1"; - MockClientSupplier mockClientSupplier = new MockClientSupplier(); StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); @@ -236,7 +232,6 @@ public void testAssignWithPartialTopology() throws Exception { props.put(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, SingleGroupPartitionGrouperStub.class); StreamsConfig config = new StreamsConfig(props); - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addProcessor("processor1", new MockProcessorSupplier(), "source1"); builder.addStateStore(new MockStateStoreSupplier("store1", false), "processor1"); @@ -249,17 +244,15 @@ public void testAssignWithPartialTopology() throws Exception { UUID uuid1 = UUID.randomUUID(); String client1 = "client1"; - StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); Map subscriptions = new HashMap<>(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, Collections.emptySet(), Collections.emptySet(), userEndPoint).encode())); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; + // will throw exception if it fails Map assignments = partitionAssignor.assign(metadata, subscriptions); @@ -275,9 +268,6 @@ public void testAssignWithPartialTopology() throws Exception { @Test public void testAssignEmptyMetadata() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); - - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); @@ -286,7 +276,7 @@ public void testAssignEmptyMetadata() throws Exception { final Set prevTasks10 = Utils.mkSet(task0); final Set standbyTasks10 = Utils.mkSet(task1); - final Cluster emptyMetadata = new Cluster("cluster", Arrays.asList(Node.noNode()), + final Cluster emptyMetadata = new Cluster("cluster", Collections.singletonList(Node.noNode()), Collections.emptySet(), Collections.emptySet(), Collections.emptySet()); @@ -295,7 +285,6 @@ public void testAssignEmptyMetadata() throws Exception { StreamThread thread10 = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); Map subscriptions = new HashMap<>(); @@ -336,9 +325,6 @@ public void testAssignEmptyMetadata() throws Exception { @Test public void testAssignWithNewTasks() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); - - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addSource("source3", "topic3"); @@ -355,12 +341,9 @@ public void testAssignWithNewTasks() throws Exception { UUID uuid2 = UUID.randomUUID(); String client1 = "client1"; - - MockClientSupplier mockClientSupplier = new MockClientSupplier(); StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); @@ -399,9 +382,7 @@ public void testAssignWithNewTasks() throws Exception { @Test public void testAssignWithStates() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); String applicationId = "test"; - TopologyBuilder builder = new TopologyBuilder(); builder.setApplicationId(applicationId); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); @@ -428,10 +409,8 @@ public void testAssignWithStates() throws Exception { String client1 = "client1"; - MockClientSupplier mockClientSupplier = new MockClientSupplier(); StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); @@ -444,9 +423,6 @@ public void testAssignWithStates() throws Exception { subscriptions.put("consumer20", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid2, Collections.emptySet(), Collections.emptySet(), userEndPoint).encode())); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; Map assignments = partitionAssignor.assign(metadata, subscriptions); // check assigned partition size: since there is no previous task and there are two sub-topologies the assignment is random so we cannot check exact match @@ -499,7 +475,6 @@ public void testAssignWithStandbyReplicas() throws Exception { props.setProperty(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, "1"); StreamsConfig config = new StreamsConfig(props); - TopologyBuilder builder = new TopologyBuilder(); builder.addSource("source1", "topic1"); builder.addSource("source2", "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); @@ -518,11 +493,9 @@ public void testAssignWithStandbyReplicas() throws Exception { UUID uuid2 = UUID.randomUUID(); String client1 = "client1"; - final MockClientSupplier mockClientSupplier = new MockClientSupplier(); StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer)); @@ -571,8 +544,6 @@ public void testAssignWithStandbyReplicas() throws Exception { @Test public void testOnAssignment() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); - TopicPartition t2p3 = new TopicPartition("topic2", 3); TopologyBuilder builder = new TopologyBuilder(); @@ -583,10 +554,9 @@ public void testOnAssignment() throws Exception { UUID uuid = UUID.randomUUID(); String client1 = "client1"; - StreamThread thread = new StreamThread(builder, config, new MockClientSupplier(), "test", client1, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), + StreamThread thread = new StreamThread(builder, config, mockClientSupplier, "test", client1, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread, "test", client1)); List activeTaskList = Utils.mkList(task0, task3); @@ -607,9 +577,7 @@ public void testOnAssignment() throws Exception { @Test public void testAssignWithInternalTopics() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); String applicationId = "test"; - TopologyBuilder builder = new TopologyBuilder(); builder.setApplicationId(applicationId); builder.addInternalTopic("topicX"); builder.addSource("source1", "topic1"); @@ -623,18 +591,16 @@ public void testAssignWithInternalTopics() throws Exception { UUID uuid1 = UUID.randomUUID(); String client1 = "client1"; - MockClientSupplier clientSupplier = new MockClientSupplier(); - StreamThread thread10 = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); - Set emptyTasks = Collections.emptySet(); + Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); @@ -647,9 +613,7 @@ public void testAssignWithInternalTopics() throws Exception { @Test public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throws Exception { - StreamsConfig config = new StreamsConfig(configProps()); String applicationId = "test"; - TopologyBuilder builder = new TopologyBuilder(); builder.setApplicationId(applicationId); builder.addInternalTopic("topicX"); builder.addSource("source1", "topic1"); @@ -666,22 +630,19 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() throw UUID uuid1 = UUID.randomUUID(); String client1 = "client1"; - MockClientSupplier clientSupplier = new MockClientSupplier(); - - StreamThread thread10 = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), + StreamThread thread10 = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(thread10, applicationId, client1)); - MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, clientSupplier.restoreConsumer); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(thread10.config, mockClientSupplier.restoreConsumer); partitionAssignor.setInternalTopicManager(internalTopicManager); Map subscriptions = new HashMap<>(); - Set emptyTasks = Collections.emptySet(); + Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer10", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, userEndPoint).encode())); - Map assignments = partitionAssignor.assign(metadata, subscriptions); + partitionAssignor.assign(metadata, subscriptions); // check prepared internal topics assertEquals(2, internalTopicManager.readyTopics.size()); @@ -693,7 +654,6 @@ public void shouldAddUserDefinedEndPointToSubscription() throws Exception { final Properties properties = configProps(); properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:8080"); final StreamsConfig config = new StreamsConfig(properties); - final TopologyBuilder builder = new TopologyBuilder(); final String applicationId = "application-id"; builder.setApplicationId(applicationId); builder.addSource("source", "input"); @@ -703,12 +663,9 @@ public void shouldAddUserDefinedEndPointToSubscription() throws Exception { final UUID uuid1 = UUID.randomUUID(); final String client1 = "client1"; - final MockClientSupplier clientSupplier = new MockClientSupplier(); - - final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); final PartitionAssignor.Subscription subscription = partitionAssignor.subscription(Utils.mkSet("input")); final SubscriptionInfo subscriptionInfo = SubscriptionInfo.decode(subscription.userData()); @@ -721,7 +678,6 @@ public void shouldMapUserEndPointToTopicPartitions() throws Exception { final String myEndPoint = "localhost:8080"; properties.put(StreamsConfig.APPLICATION_SERVER_CONFIG, myEndPoint); final StreamsConfig config = new StreamsConfig(properties); - final TopologyBuilder builder = new TopologyBuilder(); final String applicationId = "application-id"; builder.setApplicationId(applicationId); builder.addSource("source", "topic1"); @@ -733,17 +689,15 @@ public void shouldMapUserEndPointToTopicPartitions() throws Exception { final UUID uuid1 = UUID.randomUUID(); final String client1 = "client1"; - final MockClientSupplier clientSupplier = new MockClientSupplier(); - - final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); - partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, clientSupplier.restoreConsumer)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer)); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put("consumer1", new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks, myEndPoint).encode())); @@ -764,18 +718,14 @@ public void shouldThrowExceptionIfApplicationServerConfigIsNotHostPortPair() thr final StreamsConfig config = new StreamsConfig(properties); final UUID uuid1 = UUID.randomUUID(); final String client1 = "client1"; - final TopologyBuilder builder = new TopologyBuilder(); final String applicationId = "application-id"; builder.setApplicationId(applicationId); - final MockClientSupplier clientSupplier = new MockClientSupplier(); - - final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, clientSupplier.restoreConsumer)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer)); try { partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); @@ -793,18 +743,14 @@ public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() th final StreamsConfig config = new StreamsConfig(properties); final UUID uuid1 = UUID.randomUUID(); final String client1 = "client1"; - final TopologyBuilder builder = new TopologyBuilder(); final String applicationId = "application-id"; builder.setApplicationId(applicationId); - final MockClientSupplier clientSupplier = new MockClientSupplier(); - final StreamThread streamThread = new StreamThread(builder, config, clientSupplier, applicationId, client1, uuid1, + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client1, uuid1, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - try { partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client1)); Assert.fail("expected to an exception due to invalid config"); @@ -815,8 +761,7 @@ public void shouldThrowExceptionIfApplicationServerConfigPortIsNotAnInteger() th @Test public void shouldExposeHostStateToTopicPartitionsOnAssignment() throws Exception { - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - List topic = Arrays.asList(new TopicPartition("topic", 0)); + List topic = Collections.singletonList(new TopicPartition("topic", 0)); final Map> hostState = Collections.singletonMap(new HostInfo("localhost", 80), Collections.singleton(new TopicPartition("topic", 0))); @@ -829,9 +774,7 @@ public void shouldExposeHostStateToTopicPartitionsOnAssignment() throws Exceptio @Test public void shouldSetClusterMetadataOnAssignment() throws Exception { - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); - - final List topic = Arrays.asList(new TopicPartition("topic", 0)); + final List topic = Collections.singletonList(new TopicPartition("topic", 0)); final Map> hostState = Collections.singletonMap(new HostInfo("localhost", 80), Collections.singleton(new TopicPartition("topic", 0))); @@ -851,7 +794,6 @@ public void shouldSetClusterMetadataOnAssignment() throws Exception { @Test public void shouldReturnEmptyClusterMetadataIfItHasntBeenBuilt() throws Exception { - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); final Cluster cluster = partitionAssignor.clusterMetadata(); assertNotNull(cluster); } @@ -864,22 +806,47 @@ public void shouldNotLoopInfinitelyOnMissingMetadataAndShouldNotCreateRelatedTas builder.setApplicationId(applicationId); KStream stream1 = builder + + // Task 1 (should get created): .stream("topic1") + // force repartitioning for aggregation .selectKey(new KeyValueMapper() { @Override public Object apply(Object key, Object value) { return null; } }) - .through("topic2"); + .groupByKey() + + // Task 2 (should get created): + // create repartioning and changelog topic as task 1 exists + .count("count") + + // force repartitioning for join, but second join input topic unknown + // -> internal repartitioning topic should not get created + .toStream() + .map(new KeyValueMapper>() { + @Override + public KeyValue apply(Object key, Long value) { + return null; + } + }); + builder + // Task 3 (should not get created because input topic unknown) .stream("unknownTopic") + + // force repartitioning for join, but input topic unknown + // -> thus should not create internal repartitioning topic .selectKey(new KeyValueMapper() { @Override public Object apply(Object key, Object value) { return null; } }) + + // Task 4 (should not get created because input topics unknown) + // should not create any of both input repartition topics or any of both changelog topics .join( stream1, new ValueJoiner() { @@ -894,14 +861,14 @@ public Object apply(Object value1, Object value2) { final UUID uuid = UUID.randomUUID(); final String client = "client1"; - final StreamsConfig config = new StreamsConfig(configProps()); - final StreamThread streamThread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client)); + final MockInternalTopicManager mockInternalTopicManager = new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer); + partitionAssignor.setInternalTopicManager(mockInternalTopicManager); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put( client, new PartitionAssignor.Subscription( @@ -909,25 +876,26 @@ public Object apply(Object value1, Object value2) { new SubscriptionInfo(uuid, emptyTasks, emptyTasks, userEndPoint).encode() ) ); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; final Map assignment = partitionAssignor.assign(metadata, subscriptions); + final Map expectedCreatedInternalTopics = new HashMap<>(); + expectedCreatedInternalTopics.put(applicationId + "-count-repartition", 3); + expectedCreatedInternalTopics.put(applicationId + "-count-changelog", 3); + assertThat(mockInternalTopicManager.readyTopics, equalTo(expectedCreatedInternalTopics)); + final List expectedAssignment = Arrays.asList( new TopicPartition("topic1", 0), new TopicPartition("topic1", 1), new TopicPartition("topic1", 2), - new TopicPartition("topic2", 0), - new TopicPartition("topic2", 1), - new TopicPartition("topic2", 2) + new TopicPartition(applicationId + "-count-repartition", 0), + new TopicPartition(applicationId + "-count-repartition", 1), + new TopicPartition(applicationId + "-count-repartition", 2) ); - assertThat(expectedAssignment, equalTo(assignment.get(client).partitions())); + assertThat(new HashSet(assignment.get(client).partitions()), equalTo(new HashSet(expectedAssignment))); } @Test public void shouldUpdatePartitionHostInfoMapOnAssignment() throws Exception { - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); final TopicPartition partitionOne = new TopicPartition("topic", 1); final TopicPartition partitionTwo = new TopicPartition("topic", 2); final Map> firstHostState = Collections.singletonMap( @@ -945,7 +913,6 @@ public void shouldUpdatePartitionHostInfoMapOnAssignment() throws Exception { @Test public void shouldUpdateClusterMetadataOnAssignment() throws Exception { - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); final TopicPartition topicOne = new TopicPartition("topic", 1); final TopicPartition topicTwo = new TopicPartition("topic2", 2); final Map> firstHostState = Collections.singletonMap( @@ -960,15 +927,6 @@ public void shouldUpdateClusterMetadataOnAssignment() throws Exception { assertEquals(Utils.mkSet("topic", "topic2"), partitionAssignor.clusterMetadata().topics()); } - private PartitionAssignor.Assignment createAssignment(final Map> firstHostState) { - final AssignmentInfo info = new AssignmentInfo(Collections.emptyList(), - Collections.>emptyMap(), - firstHostState); - - return new PartitionAssignor.Assignment( - Collections.emptyList(), info.encode()); - } - @Test public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Exception { final Properties props = configProps(); @@ -982,13 +940,13 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Except final UUID uuid = UUID.randomUUID(); final String client = "client1"; - final StreamThread streamThread = new StreamThread(builder, config, new MockClientSupplier(), applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); + final StreamThread streamThread = new StreamThread(builder, config, mockClientSupplier, applicationId, client, uuid, new Metrics(), Time.SYSTEM, new StreamsMetadataState(builder, StreamsMetadataState.UNKNOWN_HOST), 0); - final StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); partitionAssignor.configure(config.getConsumerConfigs(streamThread, applicationId, client)); + partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamThread.config, mockClientSupplier.restoreConsumer)); final Map subscriptions = new HashMap<>(); - final Set emptyTasks = Collections.emptySet(); + final Set emptyTasks = Collections.emptySet(); subscriptions.put( "consumer1", new PartitionAssignor.Subscription( @@ -1005,9 +963,6 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Except ) ); final Set allPartitions = Utils.mkSet(t1p0, t1p1, t1p2); - // TODO: Update the code accordingly, - // This line was added to fix the test failure since internalTopicManager is created in the config method all the time. - partitionAssignor.internalTopicManager = null; final Map assign = partitionAssignor.assign(metadata, subscriptions); final PartitionAssignor.Assignment consumer1Assignment = assign.get("consumer1"); final AssignmentInfo assignmentInfo = AssignmentInfo.decode(consumer1Assignment.userData()); @@ -1020,6 +975,29 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() throws Except assertThat(allAssignedPartitions, equalTo(allPartitions)); } + @Test(expected = KafkaException.class) + public void shouldThrowKafkaExceptionIfStreamThreadNotConfigured() throws Exception { + partitionAssignor.configure(Collections.singletonMap(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1)); + } + + @Test(expected = KafkaException.class) + public void shouldThrowKafkaExceptionIfStreamThreadConfigIsNotStreamThreadInstance() throws Exception { + final Map config = new HashMap<>(); + config.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + config.put(StreamsConfig.InternalConfig.STREAM_THREAD_INSTANCE, "i am not a stream thread"); + + partitionAssignor.configure(config); + } + + private PartitionAssignor.Assignment createAssignment(final Map> firstHostState) { + final AssignmentInfo info = new AssignmentInfo(Collections.emptyList(), + Collections.>emptyMap(), + firstHostState); + + return new PartitionAssignor.Assignment( + Collections.emptyList(), info.encode()); + } + private AssignmentInfo checkAssignment(Set expectedTopics, PartitionAssignor.Assignment assignment) { // This assumed 1) DefaultPartitionGrouper is used, and 2) there is a only one topic group. @@ -1062,28 +1040,4 @@ private AssignmentInfo checkAssignment(Set expectedTopics, PartitionAssi return info; } - private class MockInternalTopicManager extends InternalTopicManager { - - Map readyTopics = new HashMap<>(); - MockConsumer restoreConsumer; - - MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { - super(new StreamsKafkaClient(streamsConfig), 0, 0); - - this.restoreConsumer = restoreConsumer; - } - - @Override - public void makeReady(InternalTopicConfig topic, int numPartitions) { - readyTopics.put(topic.name(), numPartitions); - - List partitions = new ArrayList<>(); - for (int i = 0; i < numPartitions; i++) { - partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); - } - - restoreConsumer.updatePartitions(topic.name(), partitions); - } - } - } 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 4e95911e8d4b4..7f27fc4a46ebe 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 @@ -22,6 +22,8 @@ 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.Node; +import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.record.TimestampType; @@ -39,6 +41,8 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.internals.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.streams.state.internals.ThreadCache; import org.apache.kafka.test.MockProcessorNode; import org.apache.kafka.test.MockSourceNode; @@ -59,6 +63,8 @@ import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -196,11 +202,11 @@ public void testMetrics() throws Exception { assertNotNull(metrics.getSensor(name + "-" + operation)); for (String entity : entities) { - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-avg-latency", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-latency-avg", groupName, "The average latency in milliseconds of " + entity + " " + operation + " operation.", tags))); - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-max-latency", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-latency-max", groupName, "The max latency in milliseconds of " + entity + " " + operation + " operation.", tags))); - assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-qps", groupName, + assertNotNull(metrics.metrics().get(metrics.metricName(entity + "-" + operation + "-rate", groupName, "The average number of occurrence of " + entity + " " + operation + " operation per second.", tags))); } } @@ -437,6 +443,63 @@ public void flush() { assertTrue(flushed.get()); } + + @SuppressWarnings("unchecked") + @Test + public void shouldCheckpointOffsetsOnCommit() throws Exception { + final String storeName = "test"; + final String changelogTopic = ProcessorStateManager.storeChangelogTopic("appId", storeName); + final InMemoryKeyValueStore inMemoryStore = new InMemoryKeyValueStore(storeName, null, null) { + @Override + public void init(final ProcessorContext context, final StateStore root) { + context.register(root, true, null); + } + + @Override + public boolean persistent() { + return true; + } + }; + final ProcessorTopology topology = new ProcessorTopology(Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.singletonList(inMemoryStore), + Collections.singletonMap(storeName, changelogTopic), + Collections.emptyList()); + + final TopicPartition partition = new TopicPartition(changelogTopic, 0); + final NoOpRecordCollector recordCollector = new NoOpRecordCollector() { + @Override + public Map offsets() { + + return Collections.singletonMap(partition, 543L); + } + }; + + restoreStateConsumer.updatePartitions(changelogTopic, + Collections.singletonList( + new PartitionInfo(changelogTopic, 0, null, new Node[0], new Node[0]))); + restoreStateConsumer.updateEndOffsets(Collections.singletonMap(partition, 0L)); + restoreStateConsumer.updateBeginningOffsets(Collections.singletonMap(partition, 0L)); + + final StreamsMetrics streamsMetrics = new MockStreamsMetrics(new Metrics()); + final TaskId taskId = new TaskId(0, 0); + final MockTime time = new MockTime(); + final StreamsConfig config = createConfig(baseDir); + final StreamTask streamTask = new StreamTask(taskId, "appId", partitions, topology, consumer, + restoreStateConsumer, config, streamsMetrics, + stateDirectory, new ThreadCache("testCache", 0, streamsMetrics), + time, recordCollector); + + time.sleep(config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG)); + + streamTask.commit(); + final OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(stateDirectory.directoryForTask(taskId), + ProcessorStateManager.CHECKPOINT_FILE_NAME)); + + assertThat(checkpoint.read(), equalTo(Collections.singletonMap(partition, 544L))); + } + private Iterable> records(ConsumerRecord... recs) { return Arrays.asList(recs); } 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 20d428b16ff5f..0e98f56eac000 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 @@ -438,29 +438,29 @@ public void testMetrics() throws Exception { String defaultPrefix = "thread." + thread.threadClientId(); Map defaultTags = Collections.singletonMap("client-id", thread.threadClientId()); - assertNotNull(metrics.getSensor(defaultPrefix + ".commit-time")); - assertNotNull(metrics.getSensor(defaultPrefix + ".poll-time")); - assertNotNull(metrics.getSensor(defaultPrefix + ".process-time")); - assertNotNull(metrics.getSensor(defaultPrefix + ".punctuate-time")); - assertNotNull(metrics.getSensor(defaultPrefix + ".task-creation")); - assertNotNull(metrics.getSensor(defaultPrefix + ".task-destruction")); + assertNotNull(metrics.getSensor(defaultPrefix + ".commit-latency")); + assertNotNull(metrics.getSensor(defaultPrefix + ".poll-latency")); + assertNotNull(metrics.getSensor(defaultPrefix + ".process-latency")); + assertNotNull(metrics.getSensor(defaultPrefix + ".punctuate-latency")); + assertNotNull(metrics.getSensor(defaultPrefix + ".task-created")); + assertNotNull(metrics.getSensor(defaultPrefix + ".task-closed")); assertNotNull(metrics.getSensor(defaultPrefix + ".skipped-records")); - assertNotNull(metrics.metrics().get(metrics.metricName("commit-time-avg", defaultGroupName, "The average commit time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("commit-time-max", defaultGroupName, "The maximum commit time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("commit-calls-rate", defaultGroupName, "The average per-second number of commit calls", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("poll-time-avg", defaultGroupName, "The average poll time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("poll-time-max", defaultGroupName, "The maximum poll time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("poll-calls-rate", defaultGroupName, "The average per-second number of record-poll calls", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("process-time-avg", defaultGroupName, "The average process time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("process-time-max", defaultGroupName, "The maximum process time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("process-calls-rate", defaultGroupName, "The average per-second number of process calls", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-time-avg", defaultGroupName, "The average punctuate time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-time-max", defaultGroupName, "The maximum punctuate time in ms", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-calls-rate", defaultGroupName, "The average per-second number of punctuate calls", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("task-creation-rate", defaultGroupName, "The average per-second number of newly created tasks", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("task-destruction-rate", defaultGroupName, "The average per-second number of destructed tasks", defaultTags))); - assertNotNull(metrics.metrics().get(metrics.metricName("skipped-records-count", defaultGroupName, "The average per-second number of skipped records.", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("commit-latency-avg", defaultGroupName, "The average commit time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("commit-latency-max", defaultGroupName, "The maximum commit time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("commit-rate", defaultGroupName, "The average per-second number of commit calls", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("poll-latency-avg", defaultGroupName, "The average poll time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("poll-latency-max", defaultGroupName, "The maximum poll time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("poll-rate", defaultGroupName, "The average per-second number of record-poll calls", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("process-latency-avg", defaultGroupName, "The average process time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("process-latency-max", defaultGroupName, "The maximum process time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("process-rate", defaultGroupName, "The average per-second number of process calls", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-latency-avg", defaultGroupName, "The average punctuate time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-latency-max", defaultGroupName, "The maximum punctuate time in ms", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("punctuate-rate", defaultGroupName, "The average per-second number of punctuate calls", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("task-created-rate", defaultGroupName, "The average per-second number of newly created tasks", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("task-closed-rate", defaultGroupName, "The average per-second number of closed tasks", defaultTags))); + assertNotNull(metrics.metrics().get(metrics.metricName("skipped-records-rate", defaultGroupName, "The average per-second number of skipped records.", defaultTags))); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java new file mode 100644 index 0000000000000..2fb57247e76f1 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsKafkaClientTest.java @@ -0,0 +1,42 @@ +/** + * 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 org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.streams.StreamsConfig; +import org.junit.Test; + +import java.util.Properties; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; + +public class StreamsKafkaClientTest { + + @Test + public void testConfigFromStreamsConfig() { + for (final String expectedMechanism : asList("PLAIN", "SCRAM-SHA-512")) { + final Properties props = new Properties(); + props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "some_app_id"); + props.setProperty(SaslConfigs.SASL_MECHANISM, expectedMechanism); + props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9000"); + final StreamsConfig streamsConfig = new StreamsConfig(props); + final AbstractConfig config = StreamsKafkaClient.Config.fromStreamsConfig(streamsConfig); + assertEquals(expectedMechanism, config.values().get(SaslConfigs.SASL_MECHANISM)); + assertEquals(expectedMechanism, config.getString(SaslConfigs.SASL_MECHANISM)); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java index b7c0c2cba58e7..c6bc2508f2d8c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetricsImplTest.java @@ -61,7 +61,7 @@ public void testRemoveSensor() { Sensor sensor1a = streamsMetrics.addSensor(sensorName, Sensor.RecordingLevel.DEBUG, sensor1); streamsMetrics.removeSensor(sensor1a); - Sensor sensor2 = streamsMetrics.addLatencySensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG); + Sensor sensor2 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG); streamsMetrics.removeSensor(sensor2); Sensor sensor3 = streamsMetrics.addThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG); @@ -77,7 +77,7 @@ public void testLatencyMetrics() { Map tags = new HashMap<>(); StreamsMetricsImpl streamsMetrics = new StreamsMetricsImpl(new Metrics(), groupName, tags); - Sensor sensor1 = streamsMetrics.addLatencySensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG); + Sensor sensor1 = streamsMetrics.addLatencyAndThroughputSensor(scope, entity, operation, Sensor.RecordingLevel.DEBUG); Map metrics = streamsMetrics.metrics(); // 6 metrics plus a common metric that keeps track of total registered metrics in Metrics() constructor diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index c3df49d3222ea..1f2665ac07522 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -28,7 +28,6 @@ import java.util.Set; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; @@ -171,10 +170,10 @@ public static KeyValueStoreTestDriver create(Class keyClass, Cla * @param valueDeserializer the value deserializer for the {@link ProcessorContext}; may not be null * @return the test driver; never null */ - public static KeyValueStoreTestDriver create(Serializer keySerializer, - Deserializer keyDeserializer, - Serializer valueSerializer, - Deserializer valueDeserializer) { + public static KeyValueStoreTestDriver create(final Serializer keySerializer, + final Deserializer keyDeserializer, + final Serializer valueSerializer, + final Deserializer valueDeserializer) { StateSerdes serdes = new StateSerdes("unexpected", Serdes.serdeFrom(keySerializer, keyDeserializer), Serdes.serdeFrom(valueSerializer, valueDeserializer)); @@ -203,20 +202,32 @@ protected KeyValueStoreTestDriver(final StateSerdes serdes) { this.recordCollector = new RecordCollectorImpl(producer, "KeyValueStoreTestDriver") { @SuppressWarnings("unchecked") @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - // for byte arrays we need to wrap it for comparison - - K key = serdes.keyFrom(keySerializer.serialize(record.topic(), record.key())); - V value = serdes.valueFrom(valueSerializer.serialize(record.topic(), record.value())); - - recordFlushed(key, value); + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + // for byte arrays we need to wrap it for comparison + + K keyTest = serdes.keyFrom(keySerializer.serialize(topic, key)); + V valueTest = serdes.valueFrom(valueSerializer.serialize(topic, value)); + + recordFlushed(keyTest, valueTest); } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, - StreamPartitioner partitioner) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, + StreamPartitioner partitioner) { // ignore partitioner - send(record, keySerializer, valueSerializer); + send(topic, key, value, partition, timestamp, keySerializer, valueSerializer); } }; this.stateDir = TestUtils.tempDirectory(); @@ -231,7 +242,7 @@ public void send(ProducerRecord record, Serializer keySeria - this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) { + this.context = new MockProcessorContext(this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector, null) { @Override public TaskId taskId() { return new TaskId(0, 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java index 8746a86cb44b1..a00526f24c79a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingKeyValueStoreTest.java @@ -63,7 +63,7 @@ public void setUp() throws Exception { store = new CachingKeyValueStore<>(underlyingStore, Serdes.String(), Serdes.String()); store.setFlushListener(cacheFlushListener); cache = new ThreadCache("testCache", maxCacheSizeBytes, new MockStreamsMetrics(new Metrics())); - final MockProcessorContext context = new MockProcessorContext(null, null, null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(null, null, null, (RecordCollector) null, cache); topic = "topic"; context.setRecordContext(new ProcessorRecordContext(10, 0, 0, topic)); store.init(context, null); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java index c603aa0b93fcf..e12d69386e985 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingSessionStoreTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -59,15 +60,21 @@ public class CachingSessionStoreTest { @Before public void setUp() throws Exception { underlying = new RocksDBSegmentedBytesStore("test", 60000, 3, new SessionKeySchema()); - cachingStore = new CachingSessionStore<>(underlying, + final RocksDBSessionStore sessionStore = new RocksDBSessionStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray()); + cachingStore = new CachingSessionStore<>(sessionStore, Serdes.String(), Serdes.Long()); cache = new ThreadCache("testCache", MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); - final MockProcessorContext context = new MockProcessorContext(null, TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, "topic")); cachingStore.init(context, cachingStore); } + @After + public void close() { + cachingStore.close(); + } + @Test public void shouldPutFetchFromCache() throws Exception { cachingStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); @@ -110,7 +117,7 @@ public void shouldFlushItemsToStoreOnEviction() throws Exception { assertEquals(added.size() - 1, cache.size()); final KeyValueIterator iterator = underlying.fetch(Bytes.wrap(added.get(0).key.key().getBytes()), 0, 0); final KeyValue next = iterator.next(); - assertEquals(added.get(0).key, SessionKeySerde.from(next.key.get(), Serdes.String().deserializer())); + assertEquals(added.get(0).key, SessionKeySerde.from(next.key.get(), Serdes.String().deserializer(), "topic")); assertArrayEquals(serdes.rawValue(added.get(0).value), next.value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java index 37fc9a0f6a69e..d50b551591556 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CachingWindowStoreTest.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; @@ -31,12 +32,17 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.test.MockProcessorContext; import org.apache.kafka.test.TestUtils; +import org.junit.After; import org.junit.Before; import org.junit.Test; import java.io.IOException; +import java.util.List; import static org.apache.kafka.streams.state.internals.ThreadCacheTest.memoryCacheEntrySize; +import static org.apache.kafka.test.StreamsTestUtils.toList; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -54,24 +60,31 @@ public class CachingWindowStoreTest { private String topic; private static final long DEFAULT_TIMESTAMP = 10L; private WindowStoreKeySchema keySchema; + private RocksDBWindowStore windowStore; @Before public void setUp() throws Exception { keySchema = new WindowStoreKeySchema(); underlying = new RocksDBSegmentedBytesStore("test", 30000, 3, keySchema); + windowStore = new RocksDBWindowStore<>(underlying, Serdes.Bytes(), Serdes.ByteArray(), false); cacheListener = new CachingKeyValueStoreTest.CacheFlushListenerStub<>(); - cachingStore = new CachingWindowStore<>(underlying, + cachingStore = new CachingWindowStore<>(windowStore, Serdes.String(), Serdes.String(), WINDOW_SIZE); cachingStore.setFlushListener(cacheListener); cache = new ThreadCache("testCache", MAX_CACHE_SIZE_BYTES, new MockStreamsMetrics(new Metrics())); topic = "topic"; - final MockProcessorContext context = new MockProcessorContext(null, TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), null, null, (RecordCollector) null, cache); context.setRecordContext(new ProcessorRecordContext(DEFAULT_TIMESTAMP, 0, 0, topic)); cachingStore.init(context, cachingStore); } + @After + public void closeStore() { + cachingStore.close(); + } + @Test public void shouldPutFetchFromCache() throws Exception { cachingStore.put("a", "a"); @@ -149,7 +162,7 @@ public void shouldIterateAcrossWindows() throws Exception { @Test public void shouldIterateCacheAndStore() throws Exception { final Bytes key = Bytes.wrap("1" .getBytes()); - underlying.put(Bytes.wrap(WindowStoreUtils.toBinaryKey(key, DEFAULT_TIMESTAMP, 0, WindowStoreUtils.INNER_SERDES)), "a".getBytes()); + underlying.put(Bytes.wrap(WindowStoreUtils.toBinaryKey(key, DEFAULT_TIMESTAMP, 0, WindowStoreUtils.getInnerStateSerde("topic"))), "a".getBytes()); cachingStore.put("1", "b", DEFAULT_TIMESTAMP + WINDOW_SIZE); final WindowStoreIterator fetch = cachingStore.fetch("1", DEFAULT_TIMESTAMP, DEFAULT_TIMESTAMP + WINDOW_SIZE); assertEquals(KeyValue.pair(DEFAULT_TIMESTAMP, "a"), fetch.next()); @@ -177,6 +190,18 @@ public void shouldThrowIfTryingToWriteToClosedCachingStore() throws Exception { cachingStore.put("a", "a"); } + @SuppressWarnings("unchecked") + @Test + public void shouldFetchAndIterateOverExactKeys() throws Exception { + cachingStore.put("a", "0001", 0); + cachingStore.put("aa", "0002", 0); + cachingStore.put("a", "0003", 1); + cachingStore.put("aa", "0004", 1); + cachingStore.put("a", "0005", 60000); + + final List> expected = Utils.mkList(KeyValue.pair(0L, "0001"), KeyValue.pair(1L, "0003"), KeyValue.pair(60000L, "0005")); + assertThat(toList(cachingStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected)); + } private int addItemsToCache() throws IOException { int cachedSize = 0; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.java new file mode 100644 index 0000000000000..58b42e3b53bac --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueBytesStoreTest.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.state.internals; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.test.InMemoryKeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; + +public class ChangeLoggingKeyValueBytesStoreTest { + + private final InMemoryKeyValueStore inner = new InMemoryKeyValueStore<>("kv"); + private final ChangeLoggingKeyValueBytesStore store = new ChangeLoggingKeyValueBytesStore(inner); + private final Map sent = new HashMap<>(); + private final Bytes hi = Bytes.wrap("hi".getBytes()); + private final Bytes hello = Bytes.wrap("hello".getBytes()); + private final byte[] there = "there".getBytes(); + private final byte[] world = "world".getBytes(); + + @Before + public void before() { + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + sent.put(key, value); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.Long(), + collector, + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + context.setTime(0); + store.init(context, store); + } + + @Test + public void shouldWriteKeyValueBytesToInnerStoreOnPut() throws Exception { + store.put(hi, there); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldLogChangeOnPut() throws Exception { + store.put(hi, there); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldWriteAllKeyValueToInnerStoreOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat(inner.get(hi), equalTo(there)); + assertThat(inner.get(hello), equalTo(world)); + } + + @Test + public void shouldLogChangesOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat((byte[]) sent.get(hi), equalTo(there)); + assertThat((byte[]) sent.get(hello), equalTo(world)); + } + + @Test + public void shouldPutNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(inner.get(hi), nullValue()); + } + + @Test + public void shouldReturnOldValueOnDelete() throws Exception { + store.put(hi, there); + assertThat(store.delete(hi), equalTo(there)); + } + + @Test + public void shouldLogKeyNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(sent.get(hi), nullValue()); + } + + @Test + public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat(inner.get(hi), equalTo(there)); + } + + @Test + public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat((byte[]) sent.get(hi), equalTo(there)); + } + + @Test + public void shouldReturnCurrentValueOnPutIfAbsent() throws Exception { + store.put(hi, there); + assertThat(store.putIfAbsent(hi, world), equalTo(there)); + } + + @Test + public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() throws Exception { + assertThat(store.putIfAbsent(hi, there), is(nullValue())); + } + + @Test + public void shouldReturnValueOnGetWhenExists() throws Exception { + store.put(hello, world); + assertThat(store.get(hello), equalTo(world)); + } + + @Test + public void shouldReturnNullOnGetWhenDoesntExist() throws Exception { + assertThat(store.get(hello), is(nullValue())); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java new file mode 100644 index 0000000000000..d45f82e4ba06b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingKeyValueStoreTest.java @@ -0,0 +1,216 @@ +/** + * 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 org.apache.kafka.common.metrics.Metrics; +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.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.test.InMemoryKeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.Before; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.nullValue; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; + +public class ChangeLoggingKeyValueStoreTest { + + private final InMemoryKeyValueStore inner = new InMemoryKeyValueStore<>("kv"); + private final Serde keySerde = Serdes.String(); + private final Serde valueSerde = Serdes.String(); + private final ChangeLoggingKeyValueStore store + = new ChangeLoggingKeyValueStore<>(inner, keySerde, valueSerde); + private final Map sent = new HashMap<>(); + private final String hi = "hi"; + private final Bytes hiBytes = Bytes.wrap(hi.getBytes()); + private final String there = "there"; + private final byte[] thereBytes = "there".getBytes(); + private final String hello = "hello"; + private final String world = "world"; + + @Before + public void before() { + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + sent.put(key, value); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.Long(), + collector, + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + context.setTime(0); + store.init(context, store); + } + + @Test + public void shouldWriteKeyValueBytesToInnerStoreOnPut() throws Exception { + store.put(hi, there); + assertThat(deserializedValueFromInner(hi), equalTo(there)); + } + + @Test + public void shouldLogChangeOnPut() throws Exception { + store.put(hi, there); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldWriteAllKeyValueToInnerStoreOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hello, world), + KeyValue.pair(hi, there))); + assertThat(deserializedValueFromInner(hello), equalTo(world)); + assertThat(deserializedValueFromInner(hi), equalTo(there)); + } + + @Test + public void shouldLogChangesOnPutAll() throws Exception { + store.putAll(Arrays.asList(KeyValue.pair(hi, there), + KeyValue.pair(hello, world))); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + assertThat((byte[]) sent.get(Bytes.wrap(hello.getBytes())), equalTo(world.getBytes())); + } + + @Test + public void shouldPutNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(inner.get(hiBytes), nullValue()); + } + + @Test + public void shouldReturnOldValueOnDelete() throws Exception { + store.put(hi, there); + assertThat(store.delete(hi), equalTo(there)); + } + + @Test + public void shouldReturnNullOnDeleteIfNoOldValue() throws Exception { + assertThat(store.delete(hi), is(nullValue())); + } + + @Test + public void shouldLogKeyNullOnDelete() throws Exception { + store.put(hi, there); + store.delete(hi); + assertThat(sent.get(hi), nullValue()); + } + + @Test + public void shouldWriteToInnerOnPutIfAbsentNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat(inner.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldNotWriteToInnerOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat(inner.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldWriteToChangelogOnPutIfAbsentWhenNoPreviousValue() throws Exception { + store.putIfAbsent(hi, there); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldNotWriteToChangeLogOnPutIfAbsentWhenValueForKeyExists() throws Exception { + store.put(hi, there); + store.putIfAbsent(hi, world); + assertThat((byte[]) sent.get(hiBytes), equalTo(thereBytes)); + } + + @Test + public void shouldReturnCurrentValueOnPutIfAbsent() throws Exception { + store.put(hi, there); + assertThat(store.putIfAbsent(hi, world), equalTo(there)); + } + + @Test + public void shouldReturnNullOnPutIfAbsentWhenNoPreviousValue() throws Exception { + assertThat(store.putIfAbsent(hi, there), is(nullValue())); + } + + @Test + public void shouldQueryRange() throws Exception { + store.put(hello, world); + store.put(hi, there); + store.put("zooom", "home"); + final KeyValueIterator range = store.range(hello, hi); + assertThat(range.next(), equalTo(KeyValue.pair(hello, world))); + assertThat(range.next(), equalTo(KeyValue.pair(hi, there))); + assertFalse(range.hasNext()); + } + + @Test + public void shouldReturnAllKeyValues() throws Exception { + store.put(hello, world); + store.put(hi, there); + final String zooom = "zooom"; + final String home = "home"; + store.put(zooom, home); + final KeyValueIterator all = store.all(); + assertThat(all.next(), equalTo(KeyValue.pair(hello, world))); + assertThat(all.next(), equalTo(KeyValue.pair(hi, there))); + assertThat(all.next(), equalTo(KeyValue.pair(zooom, home))); + assertFalse(all.hasNext()); + } + + @Test + public void shouldReturnValueOnGetWhenExists() throws Exception { + store.put(hello, world); + assertThat(store.get(hello), equalTo(world)); + } + + @Test + public void shouldReturnNullOnGetWhenDoesntExist() throws Exception { + assertThat(store.get(hello), is(nullValue())); + } + + @Test + public void shouldReturnInnerStoreName() throws Exception { + assertThat(store.name(), equalTo("kv")); + } + + private String deserializedValueFromInner(final String key) { + return valueSerde.deserializer().deserialize("blah", inner.get(Bytes.wrap(key.getBytes()))); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java index 621feb36c247b..6ba7b14f43623 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ChangeLoggingSegmentedBytesStoreTest.java @@ -17,7 +17,6 @@ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; @@ -48,12 +47,17 @@ public class ChangeLoggingSegmentedBytesStoreTest { public void setUp() throws Exception { final NoOpRecordCollector collector = new NoOpRecordCollector() { @Override - public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { - sent.put(record.key(), record.value()); + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + sent.put(key, value); } }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), collector, diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java index 0fd60018dd546..579f12930428c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlyKeyValueStoreTest.java @@ -59,7 +59,7 @@ public void before() { } private KeyValueStore newStoreInstance() { - return StateStoreTestUtils.newKeyValueStore(storeName, String.class, String.class); + return StateStoreTestUtils.newKeyValueStore(storeName, "app-id", String.class, String.class); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java index 8bfcb7bfb49d5..51b3bf0a4b5e6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/CompositeReadOnlySessionStoreTest.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.kstream.Windowed; -import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.apache.kafka.streams.kstream.internals.SessionWindow; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.test.ReadOnlySessionStoreStub; @@ -58,12 +58,12 @@ public void before() { @Test public void shouldFetchResulstFromUnderlyingSessionStore() throws Exception { - underlyingSessionStore.put(new Windowed<>("a", new TimeWindow(0, 0)), 1L); - underlyingSessionStore.put(new Windowed<>("a", new TimeWindow(10, 10)), 2L); + underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + underlyingSessionStore.put(new Windowed<>("a", new SessionWindow(10, 10)), 2L); final List, Long>> results = toList(sessionStore.fetch("a")); - assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new TimeWindow(0, 0)), 1L), - KeyValue.pair(new Windowed<>("a", new TimeWindow(10, 10)), 2L)), + assertEquals(Arrays.asList(KeyValue.pair(new Windowed<>("a", new SessionWindow(0, 0)), 1L), + KeyValue.pair(new Windowed<>("a", new SessionWindow(10, 10)), 2L)), results); } @@ -79,8 +79,8 @@ public void shouldFindValueForKeyWhenMultiStores() throws Exception { ReadOnlySessionStoreStub<>(); stubProviderTwo.addStore(storeName, secondUnderlying); - final Windowed keyOne = new Windowed<>("key-one", new TimeWindow(0, 0)); - final Windowed keyTwo = new Windowed<>("key-two", new TimeWindow(0, 0)); + final Windowed keyOne = new Windowed<>("key-one", new SessionWindow(0, 0)); + final Windowed keyTwo = new Windowed<>("key-two", new SessionWindow(0, 0)); underlyingSessionStore.put(keyOne, 0L); secondUnderlying.put(keyTwo, 10L); @@ -93,8 +93,8 @@ public void shouldFindValueForKeyWhenMultiStores() throws Exception { @Test public void shouldNotGetValueFromOtherStores() throws Exception { - final Windowed expectedKey = new Windowed<>("foo", new TimeWindow(0, 0)); - otherUnderlyingStore.put(new Windowed<>("foo", new TimeWindow(10, 10)), 10L); + final Windowed expectedKey = new Windowed<>("foo", new SessionWindow(0, 0)); + otherUnderlyingStore.put(new Windowed<>("foo", new SessionWindow(10, 10)), 10L); underlyingSessionStore.put(expectedKey, 1L); final KeyValueIterator, Long> result = sessionStore.fetch("foo"); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java index 50845e8bc1875..0ebdd5cac3c35 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/DelegatingPeekingKeyValueIteratorTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.streams.KeyValue; import org.apache.kafka.test.InMemoryKeyValueStore; import org.junit.Before; import org.junit.Test; @@ -37,7 +38,7 @@ public void setUp() throws Exception { } @Test - public void shouldPeekNext() throws Exception { + public void shouldPeekNextKey() throws Exception { store.put("A", "A"); final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); assertEquals("A", peekingIterator.peekNextKey()); @@ -45,6 +46,15 @@ public void shouldPeekNext() throws Exception { assertTrue(peekingIterator.hasNext()); } + @Test + public void shouldPeekNext() throws Exception { + store.put("A", "A"); + final DelegatingPeekingKeyValueIterator peekingIterator = new DelegatingPeekingKeyValueIterator<>(name, store.all()); + assertEquals(KeyValue.pair("A", "A"), peekingIterator.peekNext()); + assertEquals(KeyValue.pair("A", "A"), peekingIterator.peekNext()); + assertTrue(peekingIterator.hasNext()); + } + @Test public void shouldPeekAndIterate() throws Exception { final String[] kvs = {"a", "b", "c", "d", "e", "f"}; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java new file mode 100644 index 0000000000000..3cc217d2fbd7d --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheSessionStoreIteratorTest.java @@ -0,0 +1,113 @@ +/** + * 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 org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionKeySerde; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.KeyValueIteratorStub; +import org.junit.Test; + +import java.util.Collections; +import java.util.Iterator; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class MergedSortedCacheSessionStoreIteratorTest { + + private final String storeKey = "a"; + private final String cacheKey = "b"; + + private final SessionWindow storeWindow = new SessionWindow(0, 1); + private final Iterator, byte[]>> storeKvs = Collections.singleton( + KeyValue.pair(new Windowed<>(Bytes.wrap(storeKey.getBytes()), storeWindow), storeKey.getBytes())).iterator(); + private final SessionWindow cacheWindow = new SessionWindow(10, 20); + private final Iterator> cacheKvs = Collections.singleton(KeyValue.pair( + SessionKeySerde.toBinary( + new Windowed<>(cacheKey, cacheWindow), Serdes.String().serializer(), "topic"), new LRUCacheEntry(cacheKey.getBytes()))) + .iterator(); + + @Test + public void shouldHaveNextFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldGetNextFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + } + + @Test + public void shouldPeekNextKeyFromStore() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(storeKvs, Collections.>emptyIterator()); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(storeKey, storeWindow))); + } + + @Test + public void shouldHaveNextFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), + cacheKvs); + assertTrue(mergeIterator.hasNext()); + } + + @Test + public void shouldGetNextFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), cacheKvs); + assertThat(mergeIterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + } + + @Test + public void shouldPeekNextKeyFromCache() throws Exception { + final MergedSortedCacheSessionStoreIterator mergeIterator + = createIterator(Collections., byte[]>>emptyIterator(), cacheKvs); + assertThat(mergeIterator.peekNextKey(), equalTo(new Windowed<>(cacheKey, cacheWindow))); + } + + @Test + public void shouldIterateBothStoreAndCache() throws Exception { + final MergedSortedCacheSessionStoreIterator iterator = createIterator(storeKvs, cacheKvs); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(storeKey, storeWindow), storeKey))); + assertThat(iterator.next(), equalTo(KeyValue.pair(new Windowed<>(cacheKey, cacheWindow), cacheKey))); + assertFalse(iterator.hasNext()); + } + + private MergedSortedCacheSessionStoreIterator createIterator(final Iterator, byte[]>> storeKvs, + final Iterator> cacheKvs) { + final DelegatingPeekingKeyValueIterator, byte[]> storeIterator + = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(storeKvs)); + + final PeekingKeyValueIterator cacheIterator + = new DelegatingPeekingKeyValueIterator<>("cache", new KeyValueIteratorStub<>(cacheKvs)); + return new MergedSortedCacheSessionStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.String(), Serdes.String())); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java index b04f248e9ba89..376fca899342f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MergedSortedCacheWindowStoreIteratorTest.java @@ -19,7 +19,6 @@ import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; import org.apache.kafka.streams.state.KeyValueIterator; @@ -30,23 +29,25 @@ import java.util.ArrayList; import java.util.List; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; public class MergedSortedCacheWindowStoreIteratorTest { + private final List> windowStoreKvPairs = new ArrayList<>(); + private final ThreadCache cache = new ThreadCache("testCache", 1000000L, new MockStreamsMetrics(new Metrics())); + private final String namespace = "one"; + private final StateSerdes stateSerdes = new StateSerdes<>("foo", Serdes.String(), Serdes.String()); + @Test public void shouldIterateOverValueFromBothIterators() throws Exception { - final List> storeValues = new ArrayList<>(); - final ThreadCache cache = new ThreadCache("testCache", 1000000L, new MockStreamsMetrics(new Metrics())); - final String namespace = "one"; - final StateSerdes stateSerdes = new StateSerdes<>("foo", Serdes.String(), Serdes.String()); final List> expectedKvPairs = new ArrayList<>(); - for (long t = 0; t < 100; t += 20) { final byte[] v1Bytes = String.valueOf(t).getBytes(); - final KeyValue v1 = KeyValue.pair(Bytes.wrap(WindowStoreUtils.toBinaryKey("a", t, 0, stateSerdes)), v1Bytes); - storeValues.add(v1); + final KeyValue v1 = KeyValue.pair(t, v1Bytes); + windowStoreKvPairs.add(v1); expectedKvPairs.add(KeyValue.pair(t, v1Bytes)); final byte[] keyBytes = WindowStoreUtils.toBinaryKey("a", t + 10, 0, stateSerdes); final byte[] valBytes = String.valueOf(t + 10).getBytes(); @@ -56,11 +57,11 @@ public void shouldIterateOverValueFromBothIterators() throws Exception { byte[] binaryFrom = WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes); byte[] binaryTo = WindowStoreUtils.toBinaryKey("a", 100, 0, stateSerdes); - final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(storeValues.iterator())); + final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, binaryFrom, binaryTo); - final MergedSortedCachedWindowStoreIterator iterator = new MergedSortedCachedWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Bytes(), Serdes.ByteArray())); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Long(), Serdes.ByteArray())); int index = 0; while (iterator.hasNext()) { final KeyValue next = iterator.next(); @@ -70,4 +71,18 @@ public void shouldIterateOverValueFromBothIterators() throws Exception { } } + @Test + public void shouldPeekNextKey() throws Exception { + windowStoreKvPairs.add(KeyValue.pair(10L, "a".getBytes())); + cache.put(namespace, WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes), new LRUCacheEntry("b".getBytes())); + byte[] binaryFrom = WindowStoreUtils.toBinaryKey("a", 0, 0, stateSerdes); + byte[] binaryTo = WindowStoreUtils.toBinaryKey("a", 100, 0, stateSerdes); + final KeyValueIterator storeIterator = new DelegatingPeekingKeyValueIterator<>("name", new KeyValueIteratorStub<>(windowStoreKvPairs.iterator())); + final ThreadCache.MemoryLRUCacheBytesIterator cacheIterator = cache.range(namespace, binaryFrom, binaryTo); + final MergedSortedCacheWindowStoreIterator iterator = new MergedSortedCacheWindowStoreIterator<>(cacheIterator, storeIterator, new StateSerdes<>("name", Serdes.Long(), Serdes.ByteArray())); + assertThat(iterator.peekNextKey(), equalTo(0L)); + iterator.next(); + assertThat(iterator.peekNextKey(), equalTo(10L)); + } + } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java index 63065129f14e1..205e990a12aa9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/MeteredSegmentedBytesStoreTest.java @@ -57,7 +57,7 @@ public void setUp() throws Exception { } @Override - public Sensor addLatencySensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordLevel, String... tags) { + public Sensor addLatencyAndThroughputSensor(String scopeName, String entityName, String operationName, Sensor.RecordingLevel recordLevel, String... tags) { return metrics.sensor(operationName); } @@ -93,8 +93,7 @@ public Sensor addSensor(String name, Sensor.RecordingLevel recordLevel, Sensor.. }; - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java index 2082e00cba0bc..a2ce96c545dfb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ReadOnlyWindowStoreStub.java @@ -104,6 +104,11 @@ public void close() { } + @Override + public Long peekNextKey() { + throw new UnsupportedOperationException("peekNextKey not supported in stub"); + } + @Override public boolean hasNext() { return underlying.hasNext(); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java new file mode 100644 index 0000000000000..304772b4ec52f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplierTest.java @@ -0,0 +1,164 @@ +/** + * 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 org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBKeyValueStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + private KeyValueStore store; + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLoggingNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldReturnCachedKeyValueStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put("a", "b"); + store.put("b", "c"); + assertThat(store, is(instanceOf(CachingKeyValueStore.class))); + assertThat(cache.size(), is(2L)); + } + + @Test + public void shouldReturnMeteredStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(MeteredKeyValueStore.class))); + } + + @Test + public void shouldReturnMeteredStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(MeteredKeyValueStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + private KeyValueStore createStore(final boolean logged, final boolean cached) { + return new RocksDBKeyValueStoreSupplier<>(STORE_NAME, + Serdes.String(), + Serdes.String(), + logged, + Collections.EMPTY_MAP, + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java index 7fe490c2b82f2..e9c8f890930b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSegmentedBytesStoreTest.java @@ -62,8 +62,7 @@ public void before() { new SessionKeySchema()); stateDir = TestUtils.tempDirectory(); - final MockProcessorContext context = new MockProcessorContext(null, - stateDir, + final MockProcessorContext context = new MockProcessorContext(stateDir, Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), @@ -156,7 +155,7 @@ private byte[] serializeValue(final long value) { } private Bytes serializeKey(final Windowed key) { - return SessionKeySerde.toBinary(key, Serdes.String().serializer()); + return SessionKeySerde.toBinary(key, Serdes.String().serializer(), "topic"); } private List, Long>> toList(final KeyValueIterator iterator) { @@ -164,7 +163,7 @@ private List, Long>> toList(final KeyValueIterator next = iterator.next(); final KeyValue, Long> deserialized - = KeyValue.pair(SessionKeySerde.from(next.key.get(), Serdes.String().deserializer()), Serdes.Long().deserializer().deserialize("", next.value)); + = KeyValue.pair(SessionKeySerde.from(next.key.get(), Serdes.String().deserializer(), "topic"), Serdes.Long().deserializer().deserialize("", next.value)); results.add(deserialized); } return results; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java new file mode 100644 index 0000000000000..48618b7e7343e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreSupplierTest.java @@ -0,0 +1,178 @@ +/** + * 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 org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.kstream.Windowed; +import org.apache.kafka.streams.kstream.internals.SessionWindow; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.SessionStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBSessionStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + + private SessionStore store; + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLoggingNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldReturnCachedSessionStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put(new Windowed<>("a", new SessionWindow(0, 10)), "b"); + store.put(new Windowed<>("b", new SessionWindow(0, 10)), "c"); + assertThat(store, is(instanceOf(CachingSessionStore.class))); + assertThat(cache.size(), is(2L)); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(RocksDBSessionStore.class))); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(RocksDBSessionStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenNotLoggedOrCached() throws Exception { + store = createStore(false, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + + + private SessionStore createStore(final boolean logged, final boolean cached) { + return new RocksDBSessionStoreSupplier<>(STORE_NAME, + 10, + Serdes.String(), + Serdes.String(), + logged, + Collections.emptyMap(), + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java index 5a23a1cb6740c..1b24f8bb25678 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBSessionStoreTest.java @@ -36,6 +36,8 @@ import java.util.Arrays; import java.util.List; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -43,6 +45,7 @@ public class RocksDBSessionStoreTest { private SessionStore sessionStore; + private MockProcessorContext context; @Before public void before() { @@ -53,12 +56,11 @@ public void before() { Serdes.String(), Serdes.Long()); - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.Long(), - new NoOpRecordCollector(), - new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.Long(), + new NoOpRecordCollector(), + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); sessionStore.init(context, sessionStore); } @@ -146,6 +148,32 @@ public void shouldFindSessionsToMerge() throws Exception { assertFalse(results.hasNext()); } + @Test + public void shouldFetchExactKeys() throws Exception { + final RocksDBSegmentedBytesStore bytesStore = + new RocksDBSegmentedBytesStore("session-store", 0x7a00000000000000L, 2, new SessionKeySchema()); + + sessionStore = new RocksDBSessionStore<>(bytesStore, + Serdes.String(), + Serdes.Long()); + + sessionStore.init(context, sessionStore); + + sessionStore.put(new Windowed<>("a", new SessionWindow(0, 0)), 1L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(0, 0)), 2L); + sessionStore.put(new Windowed<>("a", new SessionWindow(10, 20)), 3L); + sessionStore.put(new Windowed<>("aa", new SessionWindow(10, 20)), 4L); + sessionStore.put(new Windowed<>("a", new SessionWindow(0x7a00000000000000L - 2, 0x7a00000000000000L - 1)), 5L); + + final KeyValueIterator, Long> iterator = sessionStore.findSessions("a", 0, Long.MAX_VALUE); + final List results = new ArrayList<>(); + while (iterator.hasNext()) { + results.add(iterator.next().value); + } + + assertThat(results, equalTo(Arrays.asList(1L, 3L, 5L))); + } + static List, Long>> toList(final KeyValueIterator, Long> iterator) { final List, Long>> results = new ArrayList<>(); while (iterator.hasNext()) { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java new file mode 100644 index 0000000000000..98fac08694ad1 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplierTest.java @@ -0,0 +1,177 @@ +/** + * 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 org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamsMetrics; +import org.apache.kafka.streams.processor.internals.MockStreamsMetrics; +import org.apache.kafka.streams.state.WindowStore; +import org.apache.kafka.test.MockProcessorContext; +import org.apache.kafka.test.NoOpRecordCollector; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsInstanceOf.instanceOf; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class RocksDBWindowStoreSupplierTest { + + private static final String STORE_NAME = "name"; + private WindowStore store; + private final ThreadCache cache = new ThreadCache("test", 1024, new MockStreamsMetrics(new Metrics())); + private final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + cache); + + @After + public void close() { + store.close(); + } + + @Test + public void shouldCreateLoggingEnabledStoreWhenWindowStoreLogged() throws Exception { + store = createStore(true, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertFalse(logged.isEmpty()); + } + + @Test + public void shouldNotBeLoggingEnabledStoreWhenLogginNotEnabled() throws Exception { + store = createStore(false, false); + final List logged = new ArrayList<>(); + final NoOpRecordCollector collector = new NoOpRecordCollector() { + @Override + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.add(new ProducerRecord(topic, partition, timestamp, key, value)); + } + }; + final MockProcessorContext context = new MockProcessorContext(TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + collector, + cache); + context.setTime(1); + store.init(context, store); + store.put("a", "b"); + assertTrue(logged.isEmpty()); + } + + @Test + public void shouldBeCachedWindowStoreWhenCachingEnabled() throws Exception { + store = createStore(false, true); + store.init(context, store); + context.setTime(1); + store.put("a", "b"); + store.put("b", "c"); + assertThat(store, is(instanceOf(CachingWindowStore.class))); + assertThat(context.getCache().size(), is(2L)); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingAndLoggingDisabled() throws Exception { + store = createStore(false, false); + assertThat(store, is(instanceOf(RocksDBWindowStore.class))); + } + + @Test + public void shouldReturnRocksDbStoreWhenCachingDisabled() throws Exception { + store = createStore(true, false); + assertThat(store, is(instanceOf(RocksDBWindowStore.class))); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenCached() throws Exception { + store = createStore(false, true); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenLogged() throws Exception { + store = createStore(true, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + @Test + public void shouldHaveMeteredStoreWhenNotLoggedOrCached() throws Exception { + store = createStore(false, false); + store.init(context, store); + final StreamsMetrics metrics = context.metrics(); + assertFalse(metrics.metrics().isEmpty()); + } + + @SuppressWarnings("unchecked") + private WindowStore createStore(final boolean logged, final boolean cached) { + return new RocksDBWindowStoreSupplier<>(STORE_NAME, + 10, + 3, + false, + Serdes.String(), + Serdes.String(), + 10, + logged, + Collections.emptyMap(), + cached).get(); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index a5225926e27ed..9f4afece8be79 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -19,11 +19,11 @@ import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Metrics; 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.utils.Bytes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.InvalidStateStoreException; @@ -52,6 +52,8 @@ import java.util.Map; import java.util.Set; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNull; @@ -95,12 +97,18 @@ public void shouldOnlyIterateOpenSegments() throws Exception { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-ShouldOnlyIterateOpenSegments") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -142,16 +150,22 @@ public void testPutAndFetch() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestPutAndFetch") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -216,16 +230,22 @@ public void testPutAndFetchBefore() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestPutAndFetchBefore") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -305,16 +325,22 @@ public void testPutAndFetchAfter() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestPutAndFetchAfter") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -392,16 +418,22 @@ public void testPutSameKeyTimestamp() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestPutSameKeyTimestamp") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -448,16 +480,22 @@ public void testCachingEnabled() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "anyTaskID") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -476,22 +514,26 @@ public void testRolling() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestRolling") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); WindowStore store = createWindowStore(context, false, true); - RocksDBWindowStore inner = - (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); try { // to validate segments final Segments segments = new Segments(windowName, retentionPeriod, numSegments); @@ -606,16 +648,22 @@ public void testRestore() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestRestore") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -655,16 +703,22 @@ public void send(ProducerRecord record, Serializer keySeria Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestRestoreII") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { changeLog.add(new KeyValue<>( - keySerializer.serialize(record.topic(), record.key()), - valueSerializer.serialize(record.topic(), record.value())) + keySerializer.serialize(topic, key), + valueSerializer.serialize(topic, value)) ); } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -706,13 +760,19 @@ public void testSegmentMaintenance() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestSegmentMaintenance") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - // do nothing + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + // do nothing } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -806,13 +866,19 @@ public void testInitialLoading() throws IOException { Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-TestInitialLoading") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - // do nothing + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + // do nothing } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -867,12 +933,18 @@ public void shouldCloseOpenIteratorsWhenStoreIsClosedAndThrowInvalidStateStoreEx Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-ShouldOnlyIterateOpenSegments") { @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { } }; MockProcessorContext context = new MockProcessorContext( - null, baseDir, + baseDir, byteArraySerde, byteArraySerde, recordCollector, cache); @@ -900,6 +972,119 @@ public void send(ProducerRecord record, Serializer keySeria } } + @SuppressWarnings("unchecked") + @Test + public void shouldFetchAndIterateOverExactKeys() throws Exception { + final File baseDir = TestUtils.tempDirectory(); + final RocksDBWindowStoreSupplier supplier = + new RocksDBWindowStoreSupplier<>( + "window", + 0x7a00000000000000L, 2, + true, + Serdes.String(), + Serdes.String(), + 0x7a00000000000000L, + true, + Collections.emptyMap(), + false); + + final Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); + final RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-ShouldOnlyIterateOpenSegments") { + @Override + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + } + }; + + final MockProcessorContext context = new MockProcessorContext( + baseDir, + byteArraySerde, byteArraySerde, + recordCollector, cache); + + final WindowStore windowStore = supplier.get(); + try { + windowStore.init(context, windowStore); + + windowStore.put("a", "0001", 0); + windowStore.put("aa", "0002", 0); + windowStore.put("a", "0003", 1); + windowStore.put("aa", "0004", 1); + windowStore.put("a", "0005", 0x7a00000000000000L - 1); + + final List expected = Utils.mkList("0001", "0003", "0005"); + assertThat(toList(windowStore.fetch("a", 0, Long.MAX_VALUE)), equalTo(expected)); + } finally { + windowStore.close(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void shouldFetchAndIterateOverExactBinaryKeys() throws Exception { + final File baseDir = TestUtils.tempDirectory(); + final RocksDBWindowStoreSupplier supplier = + new RocksDBWindowStoreSupplier<>( + "window", + 60000, 2, + true, + Serdes.Bytes(), + Serdes.String(), + 60000, + true, + Collections.emptyMap(), + false); + + final Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); + final RecordCollector recordCollector = new RecordCollectorImpl(producer, "RocksDBWindowStoreTest-ShouldOnlyIterateOpenSegments") { + @Override + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + } + }; + + final MockProcessorContext context = new MockProcessorContext( + baseDir, + byteArraySerde, byteArraySerde, + recordCollector, cache); + + final WindowStore windowStore = supplier.get(); + try { + windowStore.init(context, windowStore); + + final Bytes key1 = Bytes.wrap(new byte[]{0}); + final Bytes key2 = Bytes.wrap(new byte[]{0, 0}); + final Bytes key3 = Bytes.wrap(new byte[]{0, 0, 0}); + windowStore.put(key1, "1", 0); + windowStore.put(key2, "2", 0); + windowStore.put(key3, "3", 0); + windowStore.put(key1, "4", 1); + windowStore.put(key2, "5", 1); + windowStore.put(key3, "6", 59999); + windowStore.put(key1, "7", 59999); + windowStore.put(key2, "8", 59999); + windowStore.put(key3, "9", 59999); + + final List expectedKey1 = Utils.mkList("1", "4", "7"); + assertThat(toList(windowStore.fetch(key1, 0, Long.MAX_VALUE)), equalTo(expectedKey1)); + final List expectedKey2 = Utils.mkList("2", "5", "8"); + assertThat(toList(windowStore.fetch(key2, 0, Long.MAX_VALUE)), equalTo(expectedKey2)); + final List expectedKey3 = Utils.mkList("3", "6", "9"); + assertThat(toList(windowStore.fetch(key3, 0, Long.MAX_VALUE)), equalTo(expectedKey3)); + } finally { + windowStore.close(); + } + } + private List toList(WindowStoreIterator iterator) { ArrayList list = new ArrayList<>(); while (iterator.hasNext()) { 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 3d2da31be0c52..ae6fb5a7b6581 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 @@ -48,14 +48,16 @@ public boolean hasNext(final KeyValueIterator iterator) { } }; + private SegmentIterator iterator = null; + @Before public void before() { - final MockProcessorContext context = new MockProcessorContext(null, - TestUtils.tempDirectory(), - Serdes.String(), - Serdes.String(), - new NoOpRecordCollector(), - new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); + final MockProcessorContext context = new MockProcessorContext( + TestUtils.tempDirectory(), + Serdes.String(), + Serdes.String(), + new NoOpRecordCollector(), + new ThreadCache("testCache", 0, new MockStreamsMetrics(new Metrics()))); segmentOne.openDB(context); segmentTwo.openDB(context); segmentOne.put(Bytes.wrap("a".getBytes()), "1".getBytes()); @@ -67,13 +69,17 @@ public void before() { @After public void closeSegments() { + if (iterator != null) { + iterator.close(); + iterator = null; + } segmentOne.close(); segmentTwo.close(); } @Test public void shouldIterateOverAllSegments() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -101,7 +107,7 @@ public void shouldIterateOverAllSegments() throws Exception { @Test public void shouldOnlyIterateOverSegmentsInRange() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -121,7 +127,7 @@ public void shouldOnlyIterateOverSegmentsInRange() throws Exception { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -133,7 +139,7 @@ public void shouldThrowNoSuchElementOnPeekNextKeyIfNoNext() throws Exception { @Test(expected = NoSuchElementException.class) public void shouldThrowNoSuchElementOnNextIfNoNext() throws Exception { - final SegmentIterator iterator = new SegmentIterator( + iterator = new SegmentIterator( Arrays.asList(segmentOne, segmentTwo).iterator(), hasNextCondition, @@ -146,5 +152,4 @@ public void shouldThrowNoSuchElementOnNextIfNoNext() throws Exception { private KeyValue toStringKeyValue(final KeyValue binaryKv) { return KeyValue.pair(new String(binaryKv.key.get()), new String(binaryKv.value)); } - } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java index 47207ec149521..9e34e63062be7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SegmentsTest.java @@ -43,8 +43,7 @@ public class SegmentsTest { @Before public void createContext() { - context = new MockProcessorContext(null, - TestUtils.tempDirectory(), + context = new MockProcessorContext(TestUtils.tempDirectory(), Serdes.String(), Serdes.Long(), new NoOpRecordCollector(), diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.java new file mode 100644 index 0000000000000..8c0d2fec0f5db --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/SerializedKeyValueIteratorTest.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.streams.state.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.test.KeyValueIteratorStub; +import org.junit.Test; + +import java.util.Arrays; +import java.util.Iterator; +import java.util.NoSuchElementException; + +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +public class SerializedKeyValueIteratorTest { + + private final StateSerdes serdes = new StateSerdes<>("blah", Serdes.String(), Serdes.String()); + private final Iterator> iterator + = Arrays.asList(KeyValue.pair(Bytes.wrap("hi".getBytes()), "there".getBytes()), + KeyValue.pair(Bytes.wrap("hello".getBytes()), "world".getBytes())) + .iterator(); + private final DelegatingPeekingKeyValueIterator peeking + = new DelegatingPeekingKeyValueIterator<>("store", new KeyValueIteratorStub<>(iterator)); + private final SerializedKeyValueIterator serializedKeyValueIterator + = new SerializedKeyValueIterator<>(peeking, serdes); + + @Test + public void shouldReturnTrueOnHasNextWhenMoreResults() { + assertTrue(serializedKeyValueIterator.hasNext()); + } + + @Test + public void shouldReturnNextValueWhenItExists() throws Exception { + assertThat(serializedKeyValueIterator.next(), equalTo(KeyValue.pair("hi", "there"))); + assertThat(serializedKeyValueIterator.next(), equalTo(KeyValue.pair("hello", "world"))); + } + + @Test + public void shouldReturnFalseOnHasNextWhenNoMoreResults() throws Exception { + advanceIteratorToEnd(); + assertFalse(serializedKeyValueIterator.hasNext()); + } + + @Test + public void shouldThrowNoSuchElementOnNextWhenIteratorExhausted() throws Exception { + advanceIteratorToEnd(); + try { + serializedKeyValueIterator.next(); + fail("Expected NoSuchElementException on exhausted iterator"); + } catch (final NoSuchElementException nse) { + // pass + } + } + + @Test + public void shouldPeekNextKey() throws Exception { + assertThat(serializedKeyValueIterator.peekNextKey(), equalTo("hi")); + serializedKeyValueIterator.next(); + assertThat(serializedKeyValueIterator.peekNextKey(), equalTo("hello")); + } + + @Test(expected = UnsupportedOperationException.class) + public void shouldThrowUnsupportedOperationOnRemove() throws Exception { + serializedKeyValueIterator.remove(); + } + + private void advanceIteratorToEnd() { + serializedKeyValueIterator.next(); + serializedKeyValueIterator.next(); + } + + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java index 5fc9e1ffed26b..bb958276ebcf4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StateStoreTestUtils.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.MockProcessorContext; @@ -28,7 +29,7 @@ @SuppressWarnings("unchecked") public class StateStoreTestUtils { - public static KeyValueStore newKeyValueStore(String name, Class keyType, Class valueType) { + public static KeyValueStore newKeyValueStore(String name, String applicationId, Class keyType, Class valueType) { final InMemoryKeyValueStoreSupplier supplier = new InMemoryKeyValueStoreSupplier<>(name, null, null, @@ -37,8 +38,14 @@ public static KeyValueStore newKeyValueStore(String name, Class Collections.emptyMap()); final StateStore stateStore = supplier.get(); - stateStore.init(new MockProcessorContext(StateSerdes.withBuiltinTypes(name, keyType, valueType), - new NoOpRecordCollector()), stateStore); + stateStore.init( + new MockProcessorContext( + StateSerdes.withBuiltinTypes( + ProcessorStateManager.storeChangelogTopic(applicationId, name), + keyType, + valueType), + new NoOpRecordCollector()), + stateStore); return (KeyValueStore) stateStore; } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java index 82e524e9a0083..9b2bd538dbc9a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java @@ -23,7 +23,6 @@ import java.util.HashMap; import java.util.Map; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.internals.RecordCollectorImpl; @@ -45,15 +44,27 @@ public class StoreChangeLoggerTest { new RecordCollectorImpl(null, "StoreChangeLoggerTest") { @SuppressWarnings("unchecked") @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - logged.put((Integer) record.key(), (String) record.value()); + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + logged.put((Integer) key, (String) value); } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, - StreamPartitioner partitioner) { + public void send(final String topic, + K1 key, + V1 value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, + StreamPartitioner partitioner) { // ignore partitioner - send(record, keySerializer, valueSerializer); + send(topic, key, value, partition, timestamp, keySerializer, valueSerializer); } } ); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java index 499d82358ee2d..a228183a37a47 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/ThreadCacheTest.java @@ -105,7 +105,7 @@ private void checkOverheads(double entryFactor, double systemFactor, long desire public void cacheOverheadsSmallValues() { Runtime runtime = Runtime.getRuntime(); double factor = 0.05; - double systemFactor = 2.5; + double systemFactor = 3; // if I ask for a cache size of 10 MB, accept an overhead of 3x, i.e., 30 MBs might be allocated long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory()); int keySizeBytes = 8; int valueSizeBytes = 100; @@ -117,7 +117,7 @@ public void cacheOverheadsSmallValues() { public void cacheOverheadsLargeValues() { Runtime runtime = Runtime.getRuntime(); double factor = 0.05; - double systemFactor = 1.5; + double systemFactor = 2; // if I ask for a cache size of 10 MB, accept an overhead of 2x, i.e., 20 MBs might be allocated long desiredCacheSize = Math.min(100 * 1024 * 1024L, runtime.maxMemory()); int keySizeBytes = 8; int valueSizeBytes = 1000; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java index 708e1534dcc1e..85270cef06c6f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/WrappingStoreProviderTest.java @@ -42,9 +42,9 @@ public void before() { final StateStoreProviderStub stubProviderTwo = new StateStoreProviderStub(false); - stubProviderOne.addStore("kv", StateStoreTestUtils.newKeyValueStore("kv", String.class, String.class)); + stubProviderOne.addStore("kv", StateStoreTestUtils.newKeyValueStore("kv", "app-id", String.class, String.class)); stubProviderOne.addStore("window", new NoOpWindowStore()); - stubProviderTwo.addStore("kv", StateStoreTestUtils.newKeyValueStore("kv", String.class, String.class)); + stubProviderTwo.addStore("kv", StateStoreTestUtils.newKeyValueStore("kv", "app-id", String.class, String.class)); stubProviderTwo.addStore("window", new NoOpWindowStore()); wrappingStoreProvider = new WrappingStoreProvider( diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java new file mode 100644 index 0000000000000..9af32626d5278 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/tests/BrokerCompatibilityTest.java @@ -0,0 +1,110 @@ +/** + * 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.tests; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +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.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.test.TestUtils; + +import java.io.File; +import java.util.Collections; +import java.util.Properties; + +public class BrokerCompatibilityTest { + + private static final String SOURCE_TOPIC = "brokerCompatibilitySourceTopic"; + private static final String SINK_TOPIC = "brokerCompatibilitySinkTopic"; + + public static void main(String[] args) throws Exception { + System.out.println("StreamsTest instance started"); + + final String kafka = args.length > 0 ? args[0] : "localhost:9092"; + final String stateDirStr = args.length > 1 ? args[1] : TestUtils.tempDirectory().getAbsolutePath(); + + final File stateDir = new File(stateDirStr); + stateDir.mkdir(); + + final Properties streamsProperties = new Properties(); + streamsProperties.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + streamsProperties.put(StreamsConfig.APPLICATION_ID_CONFIG, "kafka-streams-system-test-broker-compatibility"); + streamsProperties.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); + streamsProperties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsProperties.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsProperties.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsProperties.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + + + final KStreamBuilder builder = new KStreamBuilder(); + builder.stream(SOURCE_TOPIC).to(SINK_TOPIC); + + final KafkaStreams streams = new KafkaStreams(builder, streamsProperties); + System.out.println("start Kafka Streams"); + streams.start(); + + + System.out.println("send data"); + final Properties producerProperties = new Properties(); + producerProperties.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProperties.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + producerProperties.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + + final KafkaProducer producer = new KafkaProducer<>(producerProperties); + producer.send(new ProducerRecord<>(SOURCE_TOPIC, "key", "value")); + + + System.out.println("wait for result"); + loopUntilRecordReceived(kafka); + + + System.out.println("close Kafka Streams"); + streams.close(); + } + + private static void loopUntilRecordReceived(final String kafka) { + final Properties consumerProperties = new Properties(); + consumerProperties.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + consumerProperties.put(ConsumerConfig.GROUP_ID_CONFIG, "broker-compatibility-consumer"); + consumerProperties.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + consumerProperties.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + consumerProperties.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + + final KafkaConsumer consumer = new KafkaConsumer<>(consumerProperties); + consumer.subscribe(Collections.singletonList(SINK_TOPIC)); + + while (true) { + ConsumerRecords records = consumer.poll(100); + for (ConsumerRecord record : records) { + if (record.key().equals("key") && record.value().equals("value")) { + consumer.close(); + return; + } + } + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/ShutdownDeadlockTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java similarity index 92% rename from streams/src/test/java/org/apache/kafka/streams/smoketest/ShutdownDeadlockTest.java rename to streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java index 7abbd0d415c99..e06e0346d6453 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/ShutdownDeadlockTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/ShutdownDeadlockTest.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.smoketest; +package org.apache.kafka.streams.tests; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -33,13 +33,9 @@ public class ShutdownDeadlockTest { private final String kafka; - private final String zookeeper; - - public ShutdownDeadlockTest(final String kafka, - final String zookeeper) { + public ShutdownDeadlockTest(final String kafka) { this.kafka = kafka; - this.zookeeper = zookeeper; } public void start() { @@ -47,7 +43,6 @@ public void start() { final Properties props = new Properties(); props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "shouldNotDeadlock"); props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.setProperty(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); final KStreamBuilder builder = new KStreamBuilder(); final KStream source = builder.stream(Serdes.String(), Serdes.String(), topic); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java similarity index 82% rename from streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java rename to streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java index 9f4dbd9991688..d1921261bf78b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestClient.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.kafka.streams.smoketest; +package org.apache.kafka.streams.tests; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.Aggregator; @@ -37,27 +38,35 @@ public class SmokeTestClient extends SmokeTestUtil { private final String kafka; - private final String zookeeper; private final File stateDir; private KafkaStreams streams; private Thread thread; + private boolean uncaughtException = false; - public SmokeTestClient(File stateDir, String kafka, String zookeeper) { + public SmokeTestClient(File stateDir, String kafka) { super(); this.stateDir = stateDir; this.kafka = kafka; - this.zookeeper = zookeeper; } public void start() { - streams = createKafkaStreams(stateDir, kafka, zookeeper); + streams = createKafkaStreams(stateDir, kafka); streams.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { @Override public void uncaughtException(Thread t, Throwable e) { + System.out.println("SMOKE-TEST-CLIENT-EXCEPTION"); + uncaughtException = true; e.printStackTrace(); } }); + Runtime.getRuntime().addShutdownHook(new Thread(new Runnable() { + @Override + public void run() { + close(); + } + })); + thread = new Thread() { public void run() { streams.start(); @@ -67,32 +76,38 @@ public void run() { } public void close() { - streams.close(); + streams.close(5, TimeUnit.SECONDS); + // do not remove these printouts since they are needed for health scripts + if (!uncaughtException) { + System.out.println("SMOKE-TEST-CLIENT-CLOSED"); + } try { thread.join(); } catch (Exception ex) { + // do not remove these printouts since they are needed for health scripts + System.out.println("SMOKE-TEST-CLIENT-EXCEPTION"); // ignore } } - private static KafkaStreams createKafkaStreams(File stateDir, String kafka, String zookeeper) { - Properties props = new Properties(); + private static KafkaStreams createKafkaStreams(File stateDir, String kafka) { + final Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest"); props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); props.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); - props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 2); props.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, 3); props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE); + props.put(ProducerConfig.ACKS_CONFIG, "all"); - KStreamBuilder builder = new KStreamBuilder(); + KStreamBuilder builder = new KStreamBuilder(); KStream source = builder.stream(stringSerde, intSerde, "data"); - source.to(stringSerde, intSerde, "echo"); - KStream data = source.filter(new Predicate() { @Override public boolean test(String key, Integer value) { @@ -213,7 +228,17 @@ public Double apply(Long value1, Long value2) { "cntByCnt" ).to(stringSerde, longSerde, "tagg"); - return new KafkaStreams(builder, props); + final KafkaStreams streamsClient = new KafkaStreams(builder, props); + streamsClient.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { + @Override + public void uncaughtException(Thread t, Throwable e) { + System.out.println("FATAL: An unexpected exception is encountered on thread " + t + ": " + e); + + streamsClient.close(30, TimeUnit.SECONDS); + } + }); + + return streamsClient; } } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java similarity index 96% rename from streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java rename to streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java index 2ce7e1a3ee3bc..c2cfd847ccd8b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestDriver.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streams.smoketest; +package org.apache.kafka.streams.tests; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -94,10 +94,10 @@ public void run() { } }; - SmokeTestClient streams1 = new SmokeTestClient(createDir(stateDir, "1"), kafka, zookeeper); - SmokeTestClient streams2 = new SmokeTestClient(createDir(stateDir, "2"), kafka, zookeeper); - SmokeTestClient streams3 = new SmokeTestClient(createDir(stateDir, "3"), kafka, zookeeper); - SmokeTestClient streams4 = new SmokeTestClient(createDir(stateDir, "4"), kafka, zookeeper); + SmokeTestClient streams1 = new SmokeTestClient(createDir(stateDir, "1"), kafka); + SmokeTestClient streams2 = new SmokeTestClient(createDir(stateDir, "2"), kafka); + SmokeTestClient streams3 = new SmokeTestClient(createDir(stateDir, "3"), kafka); + SmokeTestClient streams4 = new SmokeTestClient(createDir(stateDir, "4"), kafka); System.out.println("starting the driver"); driver.start(); @@ -131,13 +131,17 @@ public void run() { } public static Map> generate(String kafka, final int numKeys, final int maxRecordsPerKey) throws Exception { - Properties props = new Properties(); - props.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); - props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); - props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - - KafkaProducer producer = new KafkaProducer<>(props); + final Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, "SmokeTest"); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + // the next 4 config values make sure that all records are produced with no loss and + // no duplicates + producerProps.put(ProducerConfig.RETRIES_CONFIG, Integer.MAX_VALUE); + producerProps.put(ProducerConfig.ACKS_CONFIG, "all"); + + KafkaProducer producer = new KafkaProducer<>(producerProps); int numRecordsProduced = 0; @@ -232,7 +236,7 @@ public static void verify(String kafka, Map> allData, int m } int retry = 0; final long start = System.currentTimeMillis(); - while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(3)) { + while (System.currentTimeMillis() - start < TimeUnit.MINUTES.toMillis(6)) { ConsumerRecords records = consumer.poll(500); if (records.isEmpty() && recordsProcessed >= recordsGenerated) { if (verifyMin(min, allData, false) diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java similarity index 99% rename from streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java rename to streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java index 36660fb95e9d0..73fe27c465998 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/SmokeTestUtil.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streams.smoketest; +package org.apache.kafka.streams.tests; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/StreamsSmokeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java similarity index 84% rename from streams/src/test/java/org/apache/kafka/streams/smoketest/StreamsSmokeTest.java rename to streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java index ce0bd2b81cdae..304cae7e0ada8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/StreamsSmokeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsSmokeTest.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.kafka.streams.smoketest; +package org.apache.kafka.streams.tests; import java.io.File; import java.util.Map; @@ -30,15 +30,13 @@ public class StreamsSmokeTest { * @param args */ public static void main(String[] args) throws Exception { - String command = args[0]; - String kafka = args.length > 1 ? args[1] : null; - String zookeeper = args.length > 2 ? args[2] : null; - String stateDir = args.length > 3 ? args[3] : null; + String kafka = args[0]; + String stateDir = args.length > 1 ? args[1] : null; + String command = args.length > 2 ? args[2] : null; - System.out.println("StreamsSmokeTest instance started"); + System.out.println("StreamsTest instance started"); System.out.println("command=" + command); System.out.println("kafka=" + kafka); - System.out.println("zookeeper=" + zookeeper); System.out.println("stateDir=" + stateDir); switch (command) { @@ -54,7 +52,7 @@ public static void main(String[] args) throws Exception { break; case "process": // this starts a KafkaStreams client - final SmokeTestClient client = new SmokeTestClient(new File(stateDir), kafka, zookeeper); + final SmokeTestClient client = new SmokeTestClient(new File(stateDir), kafka); client.start(); Runtime.getRuntime().addShutdownHook(new Thread() { @@ -65,7 +63,7 @@ public void run() { }); break; case "close-deadlock-test": - final ShutdownDeadlockTest test = new ShutdownDeadlockTest(kafka, zookeeper); + final ShutdownDeadlockTest test = new ShutdownDeadlockTest(kafka); test.start(); break; default: diff --git a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java index 2f3ef26635ecf..612a0da47e6d4 100644 --- a/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java +++ b/streams/src/test/java/org/apache/kafka/test/GlobalStateManagerStub.java @@ -66,6 +66,11 @@ public void close(final Map offsets) throws IOException { closed = true; } + @Override + public void checkpoint(final Map offsets) { + this.offsets.putAll(offsets); + } + @Override public StateStore getGlobalStore(final String name) { return null; @@ -77,7 +82,7 @@ public StateStore getStore(final String name) { } @Override - public Map checkpointedOffsets() { + public Map checkpointed() { return offsets; } } diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index d51384c12bc72..dfa2987d04855 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -17,7 +17,6 @@ package org.apache.kafka.test; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; @@ -42,14 +41,11 @@ public class KStreamTestDriver { + private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L; + private final ProcessorTopology topology; private final MockProcessorContext context; private final ProcessorTopology globalTopology; - private ThreadCache cache; - private static final long DEFAULT_CACHE_SIZE_BYTES = 1 * 1024 * 1024L; - public final File stateDir; - - private ProcessorNode currNode; public KStreamTestDriver(KStreamBuilder builder) { this(builder, null, Serdes.ByteArray(), Serdes.ByteArray()); @@ -78,9 +74,8 @@ public KStreamTestDriver(KStreamBuilder builder, builder.setApplicationId("TestDriver"); this.topology = builder.build(null); this.globalTopology = builder.buildGlobalStateTopology(); - this.stateDir = stateDir; - this.cache = new ThreadCache("testCache", cacheSize, new MockStreamsMetrics(new Metrics())); - this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector(), cache); + ThreadCache cache = new ThreadCache("testCache", cacheSize, new MockStreamsMetrics(new Metrics())); + this.context = new MockProcessorContext(stateDir, keySerde, valSerde, new MockRecordCollector(), cache); this.context.setRecordContext(new ProcessorRecordContext(0, 0, 0, "topic")); // init global topology first as it will add stores to the // store map that are required for joins etc. @@ -88,7 +83,6 @@ public KStreamTestDriver(KStreamBuilder builder, initTopology(globalTopology, globalTopology.globalStateStores()); } initTopology(topology, topology.stateStores()); - } private void initTopology(final ProcessorTopology topology, final List stores) { @@ -106,14 +100,17 @@ private void initTopology(final ProcessorTopology topology, final List void forward(K key, V value) { - ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) currNode.children()) { - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - } - } - - @SuppressWarnings("unchecked") - public void forward(K key, V value, int childIndex) { - ProcessorNode thisNode = currNode; - ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - } - - @SuppressWarnings("unchecked") - public void forward(K key, V value, String childName) { - ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) thisNode.children()) { - if (childNode.name().equals(childName)) { - currNode = childNode; - try { - childNode.process(key, value); - } finally { - currNode = thisNode; - } - break; - } - } - } - public void close() { // close all processors for (ProcessorNode node : topology.processors()) { - currNode = node; + context.setCurrentNode(node); try { node.close(); } finally { - currNode = null; + context.setCurrentNode(null); } } - flushState(); + closeState(); } public Set allProcessorNames() { @@ -245,47 +195,54 @@ public void flushState() { } } - public void setCurrentNode(final ProcessorNode currentNode) { - currNode = currentNode; - } + private void closeState() { + // we need to first flush all stores before trying to close any one + // of them since the flushing could cause eviction and hence tries to access other stores + flushState(); - public StateStore globalStateStore(final String storeName) { - if (globalTopology != null) { - for (final StateStore store : globalTopology.globalStateStores()) { - if (store.name().equals(storeName)) { - return store; - } - } + for (StateStore stateStore : context.allStateStores().values()) { + stateStore.close(); } - return null; } + private ProcessorRecordContext createRecordContext(long timestamp) { + return new ProcessorRecordContext(timestamp, -1, -1, "topic"); + } private class MockRecordCollector extends RecordCollectorImpl { - public MockRecordCollector() { + MockRecordCollector() { super(null, "KStreamTestDriver"); } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, StreamPartitioner partitioner) { // The serialization is skipped. - process(record.topic(), record.key(), record.value()); + process(topic, key, value); } @Override - public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { - // The serialization is skipped. - process(record.topic(), record.key(), record.value()); + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + // The serialization is skipped. + process(topic, key, value); } @Override - public void flush() { - } + public void flush() {} @Override - public void close() { - } + public void close() {} } - } diff --git a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java index af268801dbba3..0813f3992e69e 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInternalTopicManager.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.consumer.MockConsumer; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.internals.InternalTopicConfig; import org.apache.kafka.streams.processor.internals.InternalTopicManager; @@ -28,28 +29,40 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; public class MockInternalTopicManager extends InternalTopicManager { public Map readyTopics = new HashMap<>(); - public MockConsumer restoreConsumer; + private MockConsumer restoreConsumer; public MockInternalTopicManager(StreamsConfig streamsConfig, MockConsumer restoreConsumer) { - super(new StreamsKafkaClient(streamsConfig), 0, 0); + super(new StreamsKafkaClient(streamsConfig), 0, 0, new MockTime()); this.restoreConsumer = restoreConsumer; } @Override - public void makeReady(InternalTopicConfig topic, int numPartitions) { - readyTopics.put(topic.name(), numPartitions); + public void makeReady(final Map topics) { + for (Map.Entry entry : topics.entrySet()) { + readyTopics.put(entry.getKey().name(), entry.getValue()); - List partitions = new ArrayList<>(); - for (int i = 0; i < numPartitions; i++) { - partitions.add(new PartitionInfo(topic.name(), i, null, null, null)); - } + final List partitions = new ArrayList<>(); + for (int i = 0; i < entry.getValue(); i++) { + partitions.add(new PartitionInfo(entry.getKey().name(), i, null, null, null)); + } - restoreConsumer.updatePartitions(topic.name(), partitions); + restoreConsumer.updatePartitions(entry.getKey().name(), partitions); + } } + @Override + public Map getNumPartitions(final Set topics) { + final Map partitions = new HashMap<>(); + for (String topic : topics) { + partitions.put(topic, restoreConsumer.partitionsFor(topic) == null ? null : restoreConsumer.partitionsFor(topic).size()); + } + + return partitions; + } } \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 5ae71124e3bd3..93f0f42370306 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -18,7 +18,6 @@ package org.apache.kafka.test; import java.io.File; -import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -48,7 +47,6 @@ public class MockProcessorContext implements InternalProcessorContext, RecordCollector.Supplier { - private final KStreamTestDriver driver; private final Serde keySerde; private final Serde valSerde; private final RecordCollector.Supplier recordCollectorSupplier; @@ -67,34 +65,34 @@ public class MockProcessorContext implements InternalProcessorContext, RecordCol private ProcessorNode currentNode; public MockProcessorContext(StateSerdes serdes, RecordCollector collector) { - this(null, null, serdes.keySerde(), serdes.valueSerde(), collector, null); + this(null, serdes.keySerde(), serdes.valueSerde(), collector, null); } - public MockProcessorContext(KStreamTestDriver driver, File stateDir, + public MockProcessorContext(File stateDir, Serde keySerde, Serde valSerde, final RecordCollector collector, final ThreadCache cache) { - this(driver, stateDir, keySerde, valSerde, + this(stateDir, keySerde, valSerde, new RecordCollector.Supplier() { @Override public RecordCollector recordCollector() { return collector; } - }, cache); + }, + cache); } - public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serde keySerde, - Serde valSerde, - RecordCollector.Supplier collectorSupplier, + public MockProcessorContext(final File stateDir, + final Serde keySerde, + final Serde valSerde, + final RecordCollector.Supplier collectorSupplier, final ThreadCache cache) { - this.driver = driver; this.stateDir = stateDir; this.keySerde = keySerde; this.valSerde = valSerde; this.recordCollectorSupplier = collectorSupplier; - this.metrics = new Metrics(config, Arrays.asList((MetricsReporter) new JmxReporter()), time, true); + this.metrics = new Metrics(config, Collections.singletonList((MetricsReporter) new JmxReporter()), time, true); this.cache = cache; this.streamsMetrics = new MockStreamsMetrics(metrics); } @@ -182,19 +180,45 @@ public void schedule(long interval) { @Override @SuppressWarnings("unchecked") public void forward(K key, V value) { - driver.forward(key, value); + ProcessorNode thisNode = currentNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } + } } @Override @SuppressWarnings("unchecked") public void forward(K key, V value, int childIndex) { - driver.forward(key, value, childIndex); + ProcessorNode thisNode = currentNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } } @Override @SuppressWarnings("unchecked") public void forward(K key, V value, String childName) { - driver.forward(key, value, childName); + ProcessorNode thisNode = currentNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + if (childNode.name().equals(childName)) { + currentNode = childNode; + try { + childNode.process(key, value); + } finally { + currentNode = thisNode; + } + break; + } + } } @@ -268,8 +292,7 @@ public void setRecordContext(final RecordContext recordContext) { @Override public void setCurrentNode(final ProcessorNode currentNode) { - this.currentNode = currentNode; - driver.setCurrentNode(currentNode); + this.currentNode = currentNode; } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java index 096c64a6c069c..4e0d21a062e69 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.processor.internals.SourceNode; import java.util.ArrayList; +import java.util.Arrays; import java.util.concurrent.atomic.AtomicInteger; public class MockSourceNode extends SourceNode { @@ -36,7 +37,7 @@ public class MockSourceNode extends SourceNode { public boolean initialized; public MockSourceNode(String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { - super(NAME + INDEX.getAndIncrement(), topics, keyDeserializer, valDeserializer); + super(NAME + INDEX.getAndIncrement(), Arrays.asList(topics), keyDeserializer, valDeserializer); } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java index 0fe4d63a854c3..c2833ad942642 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java +++ b/streams/src/test/java/org/apache/kafka/test/NoOpRecordCollector.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.test; -import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.StreamPartitioner; @@ -28,12 +27,25 @@ public class NoOpRecordCollector implements RecordCollector { @Override - public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer) { - // no-op + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer) { + // no-op } @Override - public void send(final ProducerRecord record, final Serializer keySerializer, final Serializer valueSerializer, final StreamPartitioner partitioner) { + public void send(final String topic, + K key, + V value, + Integer partition, + Long timestamp, + Serializer keySerializer, + Serializer valueSerializer, + StreamPartitioner partitioner) { // no-op } diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 89ca0df9e584c..7ace43a7538c9 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -204,7 +204,8 @@ public List partitionsFor(String topic) { final GlobalStateManagerImpl stateManager = new GlobalStateManagerImpl(globalTopology, globalConsumer, stateDirectory); globalStateTask = new GlobalStateUpdateTask(globalTopology, new GlobalProcessorContextImpl(config, stateManager, streamsMetrics, cache), - stateManager); + stateManager + ); globalStateTask.initialize(); } diff --git a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java index 555e622237535..7e9272a6451ff 100644 --- a/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java +++ b/streams/src/test/java/org/apache/kafka/test/StreamsTestUtils.java @@ -41,6 +41,8 @@ public static Properties getStreamsConfig(final String applicationId, streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, valueSerdeClassName); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); + streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); streamsConfiguration.putAll(additional); return streamsConfiguration; @@ -66,4 +68,5 @@ public static List> toList(final Iterator> } return results; } + } diff --git a/tests/docker/Dockerfile b/tests/docker/Dockerfile index bdd6af612dd11..78b2e32d35278 100644 --- a/tests/docker/Dockerfile +++ b/tests/docker/Dockerfile @@ -16,21 +16,16 @@ FROM openjdk:8 MAINTAINER Apache Kafka dev@kafka.apache.org -RUN apt update -RUN apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev -RUN pip install -U pip -RUN pip install --upgrade cffi -RUN pip install ducktape==0.6.0 - -VOLUME ["/kafka_src"] - +VOLUME ["/opt/kafka-dev"] ENV MIRROR="http://apache.cs.utah.edu/" -RUN wget -q "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" -O "/tmp/kafka_2.10-0.8.2.2.tgz" && tar xfz /tmp/kafka_2.10-0.8.2.2.tgz -C /opt && mv "/opt/kafka_2.10-0.8.2.2" "/opt/kafka-0.8.2.2" -RUN wget -q "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" -O "/tmp/kafka_2.10-0.9.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.9.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.9.0.1" "/opt/kafka-0.9.0.1" -RUN wget -q "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" -O "/tmp/kafka_2.10-0.10.0.1.tgz" && tar xfz /tmp/kafka_2.10-0.10.0.1.tgz -C /opt && mv "/opt/kafka_2.10-0.10.0.1" "/opt/kafka-0.10.0.1" -RUN rm /tmp/kafka_*.tgz ADD ssh /root/.ssh RUN chmod 600 /root/.ssh/id_rsa +RUN apt update && apt install -y unzip wget curl jq coreutils openssh-server net-tools vim openjdk-8-jdk python-pip python-dev libffi-dev libssl-dev +RUN pip install -U pip && pip install --upgrade cffi ducktape==0.6.0 +RUN mkdir -p "/opt/kafka-0.8.2.2" && curl -s "${MIRROR}kafka/0.8.2.2/kafka_2.10-0.8.2.2.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.8.2.2" +RUN mkdir -p "/opt/kafka-0.9.0.1" && curl -s "${MIRROR}kafka/0.9.0.1/kafka_2.10-0.9.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.9.0.1" +RUN mkdir -p "/opt/kafka-0.10.0.1" && curl -s "${MIRROR}kafka/0.10.0.1/kafka_2.10-0.10.0.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.0.1" +RUN mkdir -p "/opt/kafka-0.10.1.1" && curl -s "${MIRROR}kafka/0.10.1.1/kafka_2.10-0.10.1.1.tgz" | tar xz --strip-components=1 -C "/opt/kafka-0.10.1.1" CMD service ssh start && tail -f /dev/null diff --git a/tests/docker/run_tests.sh b/tests/docker/run_tests.sh index dadb031c251f1..11b551fad10c9 100755 --- a/tests/docker/run_tests.sh +++ b/tests/docker/run_tests.sh @@ -18,6 +18,11 @@ # TC_PATHS="tests/kafkatest/tests/streams tests/kafkatest/tests/tools" bash tests/docker/run_tests.sh set -x +die() { + echo $@ + exit 1 +} + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )" TESTS_DIR=`dirname ${SCRIPT_DIR}` KAFKA_SRC=`dirname ${TESTS_DIR}` @@ -40,13 +45,14 @@ docker run --rm -it ${KAFKA_IMAGE} "true" if [[ $? != 0 || ${KAFKA_IMAGE_REBUILD} != "" ]]; then echo "kafka image ${KAFKA_IMAGE} does not exist. Building it from scratch." COMMIT_INFO=$(git describe HEAD) - docker build -t ${KAFKA_IMAGE} --label=commit_info=${COMMIT_INFO} ${SCRIPT_DIR} + docker build -t ${KAFKA_IMAGE} --label=commit_info=${COMMIT_INFO} ${SCRIPT_DIR} \ + || die "docker build failed" fi echo "Using kafka image: ${KAFKA_IMAGE}" docker inspect ${KAFKA_IMAGE} for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do - docker run -d -t --name knode${i} --network knw -v ${KAFKA_SRC}:/kafka_src ${KAFKA_IMAGE} + docker run -d -t --name knode${i} --network knw -v ${KAFKA_SRC}:/opt/kafka-dev ${KAFKA_IMAGE} done docker info @@ -55,18 +61,8 @@ docker network inspect knw for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do echo knode${i} - docker exec knode${i} bash -c "(tar xfz /kafka_src/core/build/distributions/kafka_*SNAPSHOT.tgz -C /opt || echo missing kafka tgz did you build kafka tarball) && mv /opt/kafka*SNAPSHOT /opt/kafka-trunk && ls -l /opt" - docker exec knode01 bash -c "ssh knode$i hostname" -done - -# hack to copy test dependencies -# this is required for running MiniKDC -(cd ${KAFKA_SRC} && ./gradlew copyDependantTestLibs) -for i in $(seq -w 1 ${KAFKA_NUM_CONTAINERS}); do - echo knode${i} - docker exec knode${i} bash -c "cp /kafka_src/core/build/dependant-testlibs/* /opt/kafka-trunk/libs/" docker exec knode01 bash -c "ssh knode$i hostname" done bash tests/cluster_file_generator.sh > tests/cluster_file.json -docker exec knode01 bash -c "cd /kafka_src; ducktape ${_DUCKTAPE_OPTIONS} --cluster-file tests/cluster_file.json ${TC_PATHS:-tests/kafkatest/tests}" +docker exec knode01 bash -c "cd /opt/kafka-dev; ducktape ${_DUCKTAPE_OPTIONS} --cluster-file tests/cluster_file.json ${TC_PATHS:-tests/kafkatest/tests}" diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index 7006860f28c0a..52c820e672457 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -19,7 +19,7 @@ # due to python version naming restrictions, which are enforced by python packaging tools # (see https://www.python.org/dev/peps/pep-0440/) # -# Instead, in trunk, the version should have a suffix of the form ".devN" +# Instead, in development branches, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 0.9.0.0-SNAPSHOT, this should be something like "0.9.0.0.dev0" -__version__ = '0.10.2.0.dev0' +__version__ = '0.10.2.2.dev0' diff --git a/tests/kafkatest/benchmarks/core/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py index 14fab2f45ec65..b068fff63caf4 100644 --- a/tests/kafkatest/benchmarks/core/benchmark_test.py +++ b/tests/kafkatest/benchmarks/core/benchmark_test.py @@ -22,7 +22,7 @@ from kafkatest.services.kafka import KafkaService from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService, throughput, latency, compute_aggregate_throughput from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.version import TRUNK, KafkaVersion +from kafkatest.version import DEV_BRANCH, KafkaVersion TOPIC_REP_ONE = "topic-replication-factor-one" TOPIC_REP_THREE = "topic-replication-factor-three" @@ -72,8 +72,8 @@ def start_kafka(self, security_protocol, interbroker_security_protocol, version) @cluster(num_nodes=7) @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3) def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, - compression_type="none", security_protocol='PLAINTEXT', client_version=str(TRUNK), - broker_version=str(TRUNK)): + compression_type="none", security_protocol='PLAINTEXT', client_version=str(DEV_BRANCH), + broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor, @@ -104,8 +104,8 @@ def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DE @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) def test_long_term_producer_throughput(self, compression_type="none", security_protocol='PLAINTEXT', - interbroker_security_protocol=None, client_version=str(TRUNK), - broker_version=str(TRUNK)): + interbroker_security_protocol=None, client_version=str(DEV_BRANCH), + broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Produce 10e6 100 byte messages to a topic with 6 partitions, replication-factor 3, and acks=1. @@ -162,8 +162,8 @@ def test_long_term_producer_throughput(self, compression_type="none", security_p @cluster(num_nodes=6) @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL'], compression_type=["none", "snappy"]) def test_end_to_end_latency(self, compression_type="none", security_protocol="PLAINTEXT", - interbroker_security_protocol=None, client_version=str(TRUNK), - broker_version=str(TRUNK)): + interbroker_security_protocol=None, client_version=str(DEV_BRANCH), + broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Produce (acks = 1) and consume 10e3 messages to a topic with 6 partitions and replication-factor 3, @@ -194,7 +194,7 @@ def test_end_to_end_latency(self, compression_type="none", security_protocol="PL @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) def test_producer_and_consumer(self, compression_type="none", security_protocol="PLAINTEXT", interbroker_security_protocol=None, new_consumer=True, - client_version=str(TRUNK), broker_version=str(TRUNK)): + client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ Setup: 1 node zk + 3 node kafka cluster Concurrently produce and consume 10e6 messages with a single producer and a single consumer, @@ -243,7 +243,7 @@ def test_producer_and_consumer(self, compression_type="none", security_protocol= @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT", interbroker_security_protocol=None, new_consumer=True, num_consumers=1, - client_version=str(TRUNK), broker_version=str(TRUNK)): + client_version=str(DEV_BRANCH), broker_version=str(DEV_BRANCH)): """ Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions (using new consumer iff new_consumer == True), and report throughput. diff --git a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py index ab9b112b4fb8c..a56def3ee49af 100644 --- a/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py +++ b/tests/kafkatest/benchmarks/streams/streams_simple_benchmark_test.py @@ -13,32 +13,86 @@ # See the License for the specific language governing permissions and # limitations under the License. +from ducktape.tests.test import Test from ducktape.mark.resource import cluster - +from ducktape.mark import parametrize, matrix from kafkatest.tests.kafka_test import KafkaTest -from kafkatest.services.performance.streams_performance import StreamsSimpleBenchmarkService +from kafkatest.services.performance.streams_performance import StreamsSimpleBenchmarkService +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.version import DEV_BRANCH -class StreamsSimpleBenchmarkTest(KafkaTest): +class StreamsSimpleBenchmarkTest(Test): """ Simple benchmark of Kafka Streams. """ def __init__(self, test_context): - super(StreamsSimpleBenchmarkTest, self).__init__(test_context, num_zk=1, num_brokers=1) + super(StreamsSimpleBenchmarkTest, self).__init__(test_context) + self.num_records = 1000000L + self.replication = 1 - self.driver = StreamsSimpleBenchmarkService(test_context, self.kafka, 1000000L) - @cluster(num_nodes=3) - def test_simple_benchmark(self): + @cluster(num_nodes=9) + @matrix(test=["produce", "consume", "count", "processstream", "processstreamwithsink", "processstreamwithstatestore", "processstreamwithcachedstatestore", "kstreamktablejoin", "kstreamkstreamjoin", "ktablektablejoin"], scale=[1, 3]) + def test_simple_benchmark(self, test, scale): """ Run simple Kafka Streams benchmark """ + self.driver = [None] * (scale + 1) + node = [None] * (scale) + data = [None] * (scale) + + ############# + # SETUP PHASE + ############# + self.zk = ZookeeperService(self.test_context, num_nodes=1) + self.zk.start() + self.kafka = KafkaService(self.test_context, num_nodes=scale, zk=self.zk, version=DEV_BRANCH, topics={ + 'simpleBenchmarkSourceTopic' : { 'partitions': scale, 'replication-factor': self.replication }, + 'countTopic' : { 'partitions': scale, 'replication-factor': self.replication }, + 'simpleBenchmarkSinkTopic' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic1KStreamKStream' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic2KStreamKStream' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic1KStreamKTable' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic2KStreamKTable' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic1KTableKTable' : { 'partitions': scale, 'replication-factor': self.replication }, + 'joinSourceTopic2KTableKTable' : { 'partitions': scale, 'replication-factor': self.replication } + }) + self.kafka.start() + + ################ + # LOAD PHASE + ################ + self.load_driver = StreamsSimpleBenchmarkService(self.test_context, self.kafka, + self.num_records * scale, "true", test) + self.load_driver.start() + self.load_driver.wait() + self.load_driver.stop() + + ################ + # RUN PHASE + ################ + for num in range(0, scale): + self.driver[num] = StreamsSimpleBenchmarkService(self.test_context, self.kafka, + self.num_records/(scale), "false", test) + self.driver[num].start() - self.driver.start() - self.driver.wait() - self.driver.stop() - node = self.driver.node - node.account.ssh("grep Performance %s" % self.driver.STDOUT_FILE, allow_fail=False) + ####################### + # STOP + COLLECT PHASE + ####################### + for num in range(0, scale): + self.driver[num].wait() + self.driver[num].stop() + node[num] = self.driver[num].node + node[num].account.ssh("grep Performance %s" % self.driver[num].STDOUT_FILE, allow_fail=False) + data[num] = self.driver[num].collect_data(node[num], "" ) + - return self.driver.collect_data(node) + final = {} + for num in range(0, scale): + for key in data[num]: + final[key + str(num)] = data[num][key] + + return final diff --git a/tests/kafkatest/directory_layout/kafka_path.py b/tests/kafkatest/directory_layout/kafka_path.py index 0e60affa2426a..ece8be58214e0 100644 --- a/tests/kafkatest/directory_layout/kafka_path.py +++ b/tests/kafkatest/directory_layout/kafka_path.py @@ -16,7 +16,7 @@ import importlib import os -from kafkatest.version import get_version, KafkaVersion, TRUNK +from kafkatest.version import get_version, KafkaVersion, DEV_BRANCH """This module serves a few purposes: @@ -43,7 +43,7 @@ TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME = "tools-dependant-libs" JARS = { - "trunk": { + "dev": { CORE_JAR_NAME: "core/build/*/*.jar", CORE_LIBS_JAR_NAME: "core/build/libs/*.jar", CORE_DEPENDANT_TEST_LIBS_JAR_NAME: "core/build/dependant-testlibs/*.jar", @@ -97,7 +97,7 @@ def path(self): class KafkaSystemTestPathResolver(object): """Path resolver for Kafka system tests which assumes the following layout: - /opt/kafka-trunk # Current version of kafka under test + /opt/kafka-dev # Current version of kafka under test /opt/kafka-0.9.0.1 # Example of an older version of kafka installed from tarball /opt/kafka- # Other previous versions of kafka ... @@ -106,7 +106,7 @@ def __init__(self, context, project="kafka"): self.context = context self.project = project - def home(self, node_or_version=TRUNK): + def home(self, node_or_version=DEV_BRANCH): version = self._version(node_or_version) home_dir = self.project if version is not None: @@ -114,15 +114,15 @@ def home(self, node_or_version=TRUNK): return os.path.join(KAFKA_INSTALL_ROOT, home_dir) - def bin(self, node_or_version=TRUNK): + def bin(self, node_or_version=DEV_BRANCH): version = self._version(node_or_version) return os.path.join(self.home(version), "bin") - def script(self, script_name, node_or_version=TRUNK): + def script(self, script_name, node_or_version=DEV_BRANCH): version = self._version(node_or_version) return os.path.join(self.bin(version), script_name) - def jar(self, jar_name, node_or_version=TRUNK): + def jar(self, jar_name, node_or_version=DEV_BRANCH): version = self._version(node_or_version) return os.path.join(self.home(version), JARS[str(version)][jar_name]) diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py index 38db057e5654a..066d6d42c1477 100644 --- a/tests/kafkatest/sanity_checks/test_console_consumer.py +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -47,7 +47,7 @@ def setUp(self): @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @matrix(security_protocol=['PLAINTEXT', 'SSL']) @cluster(num_nodes=4) - @parametrize(security_protocol='SASL_SSL', sasl_mechanism='PLAIN') + @matrix(security_protocol=['SASL_SSL'], sasl_mechanism=['PLAIN', 'SCRAM-SHA-256', 'SCRAM-SHA-512']) @matrix(security_protocol=['SASL_PLAINTEXT', 'SASL_SSL']) def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='GSSAPI'): """Check that console consumer starts/stops properly, and that we are capturing log output.""" diff --git a/tests/kafkatest/sanity_checks/test_kafka_version.py b/tests/kafkatest/sanity_checks/test_kafka_version.py index 35500935c69f4..7e65807270e5f 100644 --- a/tests/kafkatest/sanity_checks/test_kafka_version.py +++ b/tests/kafkatest/sanity_checks/test_kafka_version.py @@ -19,7 +19,7 @@ from kafkatest.services.kafka import KafkaService, config_property from kafkatest.services.zookeeper import ZookeeperService from kafkatest.utils import is_version -from kafkatest.version import LATEST_0_8_2, TRUNK +from kafkatest.version import LATEST_0_8_2, DEV_BRANCH class KafkaVersionTest(Test): @@ -47,12 +47,12 @@ def test_0_8_2(self): @cluster(num_nodes=3) def test_multi_version(self): """Test kafka service node-versioning api - ensure we can bring up a 2-node cluster, one on version 0.8.2.X, - the other on trunk.""" + the other on the current development branch.""" self.kafka = KafkaService(self.test_context, num_nodes=2, zk=self.zk, topics={self.topic: {"partitions": 1, "replication-factor": 2}}) self.kafka.nodes[1].version = LATEST_0_8_2 self.kafka.nodes[1].config[config_property.INTER_BROKER_PROTOCOL_VERSION] = "0.8.2.X" self.kafka.start() - assert is_version(self.kafka.nodes[0], [TRUNK.vstring]) + assert is_version(self.kafka.nodes[0], [DEV_BRANCH.vstring]) assert is_version(self.kafka.nodes[1], [LATEST_0_8_2]) diff --git a/tests/kafkatest/sanity_checks/test_performance_services.py b/tests/kafkatest/sanity_checks/test_performance_services.py index b939f2b61ce98..6ff34b8606bee 100644 --- a/tests/kafkatest/sanity_checks/test_performance_services.py +++ b/tests/kafkatest/sanity_checks/test_performance_services.py @@ -21,7 +21,7 @@ from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService from kafkatest.services.performance import latency, compute_aggregate_throughput from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.version import TRUNK, LATEST_0_8_2, LATEST_0_9, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, KafkaVersion class PerformanceServiceTest(Test): @@ -42,8 +42,8 @@ def setUp(self): @parametrize(version=str(LATEST_0_8_2), new_consumer=False) @parametrize(version=str(LATEST_0_9), new_consumer=False) @parametrize(version=str(LATEST_0_9)) - @parametrize(version=str(TRUNK), new_consumer=False) - @parametrize(version=str(TRUNK)) + @parametrize(version=str(DEV_BRANCH), new_consumer=False) + @parametrize(version=str(DEV_BRANCH)) def test_version(self, version=str(LATEST_0_9), new_consumer=True): """ Sanity check out producer performance service - verify that we can run the service with a small diff --git a/tests/kafkatest/sanity_checks/test_verifiable_producer.py b/tests/kafkatest/sanity_checks/test_verifiable_producer.py index 544d7b989193f..be1057471948d 100644 --- a/tests/kafkatest/sanity_checks/test_verifiable_producer.py +++ b/tests/kafkatest/sanity_checks/test_verifiable_producer.py @@ -23,7 +23,7 @@ from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.zookeeper import ZookeeperService from kafkatest.utils import is_version -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, TRUNK, KafkaVersion +from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, DEV_BRANCH, KafkaVersion class TestVerifiableProducer(Test): @@ -48,10 +48,12 @@ def setUp(self): @cluster(num_nodes=3) @parametrize(producer_version=str(LATEST_0_8_2)) @parametrize(producer_version=str(LATEST_0_9)) - @parametrize(producer_version=str(TRUNK)) - def test_simple_run(self, producer_version=TRUNK): + @parametrize(producer_version=str(LATEST_0_10_0)) + @parametrize(producer_version=str(LATEST_0_10_1)) + @parametrize(producer_version=str(DEV_BRANCH)) + def test_simple_run(self, producer_version=DEV_BRANCH): """ - Test that we can start VerifiableProducer on trunk or against the 0.8.2 jar, and + Test that we can start VerifiableProducer on the current branch snapshot version or against the 0.8.2 jar, and verify that we can produce a small number of messages. """ node = self.producer.nodes[0] @@ -61,11 +63,11 @@ def test_simple_run(self, producer_version=TRUNK): err_msg="Producer failed to start in a reasonable amount of time.") # using version.vstring (distutils.version.LooseVersion) is a tricky way of ensuring - # that this check works with TRUNK - # When running VerifiableProducer 0.8.X, both trunk version and 0.8.X should show up because of the way - # verifiable producer pulls in some trunk directories into its classpath + # that this check works with DEV_BRANCH + # When running VerifiableProducer 0.8.X, both the current branch version and 0.8.X should show up because of the + # way verifiable producer pulls in some development directories into its classpath if node.version <= LATEST_0_8_2: - assert is_version(node, [node.version.vstring, TRUNK.vstring]) + assert is_version(node, [node.version.vstring, DEV_BRANCH.vstring]) else: assert is_version(node, [node.version.vstring]) diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index 5a8581a86c559..a27acd7694358 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -272,7 +272,7 @@ def start_node(self, node): cmd = self.start_cmd(node) self.logger.debug("Connect distributed command: %s", cmd) node.account.ssh(cmd) - monitor.wait_until('Kafka Connect started', timeout_sec=15, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) + monitor.wait_until('Kafka Connect started', timeout_sec=30, err_msg="Never saw message indicating Kafka Connect finished startup on " + str(node.account)) if len(self.pids(node)) == 0: raise RuntimeError("No process ids recorded") diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index 17ddb6be26aa6..d55d012e7f491 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -21,7 +21,7 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.services.monitor.jmx import JmxMixin -from kafkatest.version import TRUNK, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_10_0_0 +from kafkatest.version import DEV_BRANCH, LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, V_0_10_0_0 """ 0.8.2.1 ConsoleConsumer options @@ -73,6 +73,8 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) LOG_FILE = os.path.join(LOG_DIR, "console_consumer.log") LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "console_consumer.properties") + JMX_TOOL_LOG = "/mnt/jmx_tool.log" + JMX_TOOL_ERROR_LOG = "/mnt/jmx_tool.err.log" logs = { "consumer_stdout": { @@ -83,11 +85,17 @@ class ConsoleConsumer(KafkaPathResolverMixin, JmxMixin, BackgroundThreadService) "collect_default": False}, "consumer_log": { "path": LOG_FILE, - "collect_default": True} + "collect_default": True}, + "jmx_log": { + "path" : JMX_TOOL_LOG, + "collect_default": False}, + "jmx_err_log": { + "path": JMX_TOOL_ERROR_LOG, + "collect_default": False} } def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-group", new_consumer=True, - message_validator=None, from_beginning=True, consumer_timeout_ms=None, version=TRUNK, + message_validator=None, from_beginning=True, consumer_timeout_ms=None, version=DEV_BRANCH, client_id="console-consumer", print_key=False, jmx_object_names=None, jmx_attributes=None, enable_systest_events=False, stop_timeout_sec=15, print_timestamp=False): """ @@ -150,7 +158,8 @@ def prop_file(self, node): # Add security properties to the config. If security protocol is not specified, # use the default in the template properties. - self.security_config = self.kafka.security_config.client_config(prop_file) + self.security_config = self.kafka.security_config.client_config(prop_file, node) + self.security_config.setup_node(node) prop_file += str(self.security_config) return prop_file @@ -231,7 +240,6 @@ def _worker(self, idx, node): prop_file = self.prop_file(node) self.logger.info(prop_file) node.account.create_file(ConsoleConsumer.CONFIG_FILE, prop_file) - self.security_config.setup_node(node) # Create and upload log properties log_config = self.render('tools_log4j.properties', log_file=ConsoleConsumer.LOG_FILE) @@ -245,11 +253,12 @@ def _worker(self, idx, node): first_line = next(consumer_output, None) if first_line is not None: + self.logger.debug("collecting following jmx objects: %s", self.jmx_object_names) + self.init_jmx_attributes() self.start_jmx_tool(idx, node) for line in itertools.chain([first_line], consumer_output): msg = line.strip() - if msg == "shutdown_complete": # Note that we can only rely on shutdown_complete message if running 0.10.0 or greater if node in self.clean_shutdown_nodes: @@ -281,3 +290,24 @@ def clean_node(self, node): node.account.kill_process("java", clean_shutdown=False, allow_fail=True) node.account.ssh("rm -rf %s" % ConsoleConsumer.PERSISTENT_ROOT, allow_fail=False) self.security_config.clean_node(node) + + def has_partitions_assigned(self, node): + if self.new_consumer is False: + return False + idx = self.idx(node) + self.init_jmx_attributes() + self.start_jmx_tool(idx, node) + self.read_jmx_output(idx, node) + if not self.assigned_partitions_jmx_attr in self.maximum_jmx_value: + return False + self.logger.debug("Number of partitions assigned %f" % self.maximum_jmx_value[self.assigned_partitions_jmx_attr]) + return self.maximum_jmx_value[self.assigned_partitions_jmx_attr] > 0.0 + + def init_jmx_attributes(self): + if self.new_consumer is True: + if self.jmx_object_names is None: + self.jmx_object_names = [] + self.jmx_object_names += ["kafka.consumer:type=consumer-coordinator-metrics,client-id=%s" % self.client_id] + self.jmx_attributes += ["assigned-partitions"] + self.assigned_partitions_jmx_attr = "kafka.consumer:type=consumer-coordinator-metrics,client-id=%s:assigned-partitions" % self.client_id + diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 716c2d2645402..7e4a5899e6c48 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -30,7 +30,7 @@ from kafkatest.services.monitor.jmx import JmxMixin from kafkatest.services.security.minikdc import MiniKdc from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import TRUNK +from kafkatest.version import DEV_BRANCH Port = collections.namedtuple('Port', ['name', 'number', 'open']) @@ -67,7 +67,7 @@ class KafkaService(KafkaPathResolverMixin, JmxMixin, Service): def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT, client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, - authorizer_class_name=None, topics=None, version=TRUNK, jmx_object_names=None, + authorizer_class_name=None, topics=None, version=DEV_BRANCH, jmx_object_names=None, jmx_attributes=None, zk_connect_timeout=5000, zk_session_timeout=6000, server_prop_overides=[]): """ :type context @@ -126,9 +126,14 @@ def set_version(self, version): @property def security_config(self): - return SecurityConfig(self.context, self.security_protocol, self.interbroker_security_protocol, + config = SecurityConfig(self.context, self.security_protocol, self.interbroker_security_protocol, zk_sasl=self.zk.zk_sasl, client_sasl_mechanism=self.client_sasl_mechanism, interbroker_sasl_mechanism=self.interbroker_sasl_mechanism) + for protocol in self.port_mappings: + port = self.port_mappings[protocol] + if port.open: + config.enable_security_protocol(port.name) + return config def open_port(self, protocol): self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=True) @@ -208,6 +213,7 @@ def start_node(self, node): node.account.create_file(self.LOG4J_CONFIG, self.render('log4j.properties', log_dir=KafkaService.OPERATIONAL_LOG_DIR)) self.security_config.setup_node(node) + self.security_config.setup_credentials(node, self.path, self.zk.connect_setting(), broker=True) cmd = self.start_cmd(node) self.logger.debug("Attempting to start KafkaService on %s with command: %s" % (str(node.account), cmd)) @@ -215,6 +221,11 @@ def start_node(self, node): node.account.ssh(cmd) monitor.wait_until("Kafka Server.*started", timeout_sec=30, backoff_sec=.25, err_msg="Kafka server didn't finish startup") + # Credentials for inter-broker communication are created before starting Kafka. + # Client credentials are created after starting Kafka so that both loading of + # existing credentials from ZK and dynamic update of credentials in Kafka are tested. + self.security_config.setup_credentials(node, self.path, self.zk.connect_setting(), broker=False) + self.start_jmx_tool(self.idx(node), node) if len(self.pids(node)) == 0: raise Exception("No process ids recorded on node %s" % str(node)) diff --git a/tests/kafkatest/services/monitor/jmx.py b/tests/kafkatest/services/monitor/jmx.py index e71040bb5de30..e64d03afd48d7 100644 --- a/tests/kafkatest/services/monitor/jmx.py +++ b/tests/kafkatest/services/monitor/jmx.py @@ -35,6 +35,7 @@ def __init__(self, num_nodes, jmx_object_names=None, jmx_attributes=None): self.average_jmx_value = {} # map from object_attribute_name to average value observed over time self.jmx_tool_log = "/mnt/jmx_tool.log" + self.jmx_tool_err_log = "/mnt/jmx_tool.err.log" def clean_node(self, node): node.account.kill_process("jmx", clean_shutdown=False, allow_fail=True) @@ -55,11 +56,12 @@ def start_jmx_tool(self, idx, node): cmd += " --object-name %s" % jmx_object_name for jmx_attribute in self.jmx_attributes: cmd += " --attributes %s" % jmx_attribute - cmd += " >> %s &" % self.jmx_tool_log + cmd += " 1>> %s" % self.jmx_tool_log + cmd += " 2>> %s &" % self.jmx_tool_err_log self.logger.debug("%s: Start JmxTool %d command: %s" % (node.account, idx, cmd)) node.account.ssh(cmd, allow_fail=False) - wait_until(lambda: self._jmx_has_output(node), timeout_sec=5, backoff_sec=.5, err_msg="%s: Jmx tool took too long to start" % node.account) + wait_until(lambda: self._jmx_has_output(node), timeout_sec=10, backoff_sec=.5, err_msg="%s: Jmx tool took too long to start" % node.account) self.started[idx-1] = True def _jmx_has_output(self, node): diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py index 3e02a5ba2fbac..e1cd3a0e1b834 100644 --- a/tests/kafkatest/services/performance/consumer_performance.py +++ b/tests/kafkatest/services/performance/consumer_performance.py @@ -18,7 +18,7 @@ from kafkatest.services.performance import PerformanceService from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import TRUNK, V_0_9_0_0, LATEST_0_10_0 +from kafkatest.version import DEV_BRANCH, V_0_9_0_0, LATEST_0_10_0 class ConsumerPerformanceService(PerformanceService): @@ -70,7 +70,7 @@ class ConsumerPerformanceService(PerformanceService): "collect_default": True} } - def __init__(self, context, num_nodes, kafka, topic, messages, version=TRUNK, new_consumer=True, settings={}): + def __init__(self, context, num_nodes, kafka, topic, messages, version=DEV_BRANCH, new_consumer=True, settings={}): super(ConsumerPerformanceService, self).__init__(context, num_nodes) self.kafka = kafka self.security_config = kafka.security_config.client_config() diff --git a/tests/kafkatest/services/performance/end_to_end_latency.py b/tests/kafkatest/services/performance/end_to_end_latency.py index 917ac85bc5dce..bdaf5502c751e 100644 --- a/tests/kafkatest/services/performance/end_to_end_latency.py +++ b/tests/kafkatest/services/performance/end_to_end_latency.py @@ -17,7 +17,7 @@ from kafkatest.services.performance import PerformanceService from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import TRUNK, V_0_9_0_0 +from kafkatest.version import DEV_BRANCH, V_0_9_0_0 @@ -45,7 +45,7 @@ class EndToEndLatencyService(PerformanceService): "collect_default": True} } - def __init__(self, context, num_nodes, kafka, topic, num_records, compression_type="none", version=TRUNK, acks=1): + def __init__(self, context, num_nodes, kafka, topic, num_records, compression_type="none", version=DEV_BRANCH, acks=1): super(EndToEndLatencyService, self).__init__(context, num_nodes) self.kafka = kafka self.security_config = kafka.security_config.client_config() diff --git a/tests/kafkatest/services/performance/producer_performance.py b/tests/kafkatest/services/performance/producer_performance.py index 1113e0db65ca0..ff92da86bbf06 100644 --- a/tests/kafkatest/services/performance/producer_performance.py +++ b/tests/kafkatest/services/performance/producer_performance.py @@ -22,7 +22,7 @@ from kafkatest.services.monitor.jmx import JmxMixin from kafkatest.services.performance import PerformanceService from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import TRUNK, V_0_9_0_0 +from kafkatest.version import DEV_BRANCH, V_0_9_0_0 class ProducerPerformanceService(JmxMixin, PerformanceService): @@ -34,7 +34,7 @@ class ProducerPerformanceService(JmxMixin, PerformanceService): LOG_FILE = os.path.join(LOG_DIR, "producer_performance.log") LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") - def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, version=TRUNK, settings=None, + def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, version=DEV_BRANCH, settings=None, intermediate_stats=False, client_id="producer-performance", jmx_object_names=None, jmx_attributes=None): JmxMixin.__init__(self, num_nodes, jmx_object_names, jmx_attributes or []) @@ -89,11 +89,11 @@ def start_cmd(self, node): cmd = "" - if node.version < TRUNK: + if node.version < DEV_BRANCH: # In order to ensure more consistent configuration between versions, always use the ProducerPerformance - # tool from trunk - tools_jar = self.path.jar(TOOLS_JAR_NAME, TRUNK) - tools_dependant_libs_jar = self.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, TRUNK) + # tool from the development branch + tools_jar = self.path.jar(TOOLS_JAR_NAME, DEV_BRANCH) + tools_dependant_libs_jar = self.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, DEV_BRANCH) cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_jar cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_dependant_libs_jar diff --git a/tests/kafkatest/services/performance/streams_performance.py b/tests/kafkatest/services/performance/streams_performance.py index 0af13f9b349ca..8cedb51a21892 100644 --- a/tests/kafkatest/services/performance/streams_performance.py +++ b/tests/kafkatest/services/performance/streams_performance.py @@ -13,115 +13,28 @@ # See the License for the specific language governing permissions and # limitations under the License. -import os.path -import signal +from kafkatest.services.streams import StreamsTestBaseService -from ducktape.services.service import Service -from ducktape.utils.util import wait_until - -from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin # # Class used to start the simple Kafka Streams benchmark # -class StreamsSimpleBenchmarkService(KafkaPathResolverMixin, Service): +class StreamsSimpleBenchmarkService(StreamsTestBaseService): """Base class for simple Kafka Streams benchmark""" - PERSISTENT_ROOT = "/mnt/streams" - # The log file contains normal log4j logs written using a file appender. stdout and stderr are handled separately - LOG_FILE = os.path.join(PERSISTENT_ROOT, "streams.log") - STDOUT_FILE = os.path.join(PERSISTENT_ROOT, "streams.stdout") - STDERR_FILE = os.path.join(PERSISTENT_ROOT, "streams.stderr") - LOG4J_CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") - PID_FILE = os.path.join(PERSISTENT_ROOT, "streams.pid") - - logs = { - "streams_log": { - "path": LOG_FILE, - "collect_default": True}, - "streams_stdout": { - "path": STDOUT_FILE, - "collect_default": True}, - "streams_stderr": { - "path": STDERR_FILE, - "collect_default": True}, - } - - def __init__(self, context, kafka, numrecs): - super(StreamsSimpleBenchmarkService, self).__init__(context, 1) - self.kafka = kafka - self.numrecs = numrecs - - @property - def node(self): - return self.nodes[0] - - def pids(self, node): - try: - return [pid for pid in node.account.ssh_capture("cat " + self.PID_FILE, callback=int)] - except: - return [] - - def stop_node(self, node, clean_shutdown=True): - self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping SimpleBenchmark on " + str(node.account)) - pids = self.pids(node) - sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL - - for pid in pids: - node.account.signal(pid, sig, allow_fail=True) - if clean_shutdown: - for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="SimpleBenchmark process on " + str(node.account) + " took too long to exit") - - node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) - - def wait(self): - for node in self.nodes: - for pid in self.pids(node): - wait_until(lambda: not node.account.alive(pid), timeout_sec=600, backoff_sec=1, err_msg="SimpleBenchmark process on " + str(node.account) + " took too long to exit") - - def clean_node(self, node): - node.account.kill_process("streams", clean_shutdown=False, allow_fail=True) - node.account.ssh("rm -rf " + self.PERSISTENT_ROOT, allow_fail=False) - - def start_cmd(self, node): - args = {} - args['kafka'] = self.kafka.bootstrap_servers() - args['zk'] = self.kafka.zk.connect_setting() - args['state_dir'] = self.PERSISTENT_ROOT - args['numrecs'] = self.numrecs - args['stdout'] = self.STDOUT_FILE - args['stderr'] = self.STDERR_FILE - args['pidfile'] = self.PID_FILE - args['log4j'] = self.LOG4J_CONFIG_FILE - args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) - - cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.perf.SimpleBenchmark " \ - " %(kafka)s %(zk)s %(state_dir)s %(numrecs)s " \ - " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args - - return cmd - - def start_node(self, node): - node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) - - node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('tools_log4j.properties', log_file=self.LOG_FILE)) - - self.logger.info("Starting SimpleBenchmark process on " + str(node.account)) - results = {} - with node.account.monitor_log(self.STDOUT_FILE) as monitor: - node.account.ssh(self.start_cmd(node)) - monitor.wait_until('SimpleBenchmark instance started', timeout_sec=15, err_msg="Never saw message indicating SimpleBenchmark finished startup on " + str(node.account)) - - if len(self.pids(node)) == 0: - raise RuntimeError("No process ids recorded") + def __init__(self, test_context, kafka, numrecs, load_phase, test_name): + super(StreamsSimpleBenchmarkService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.perf.SimpleBenchmark", + numrecs, + load_phase, + test_name) - def collect_data(self, node): + def collect_data(self, node, tag = None): # Collect the data and return it to the framework output = node.account.ssh_capture("grep Performance %s" % self.STDOUT_FILE) data = {} for line in output: parts = line.split(':') - data[parts[0]] = float(parts[1]) + data[tag + parts[0]] = parts[1] return data diff --git a/tests/kafkatest/services/security/minikdc.py b/tests/kafkatest/services/security/minikdc.py index b3cbeaef53060..86a713882b010 100644 --- a/tests/kafkatest/services/security/minikdc.py +++ b/tests/kafkatest/services/security/minikdc.py @@ -24,7 +24,7 @@ from ducktape.services.service import Service from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin, CORE_LIBS_JAR_NAME, CORE_DEPENDANT_TEST_LIBS_JAR_NAME -from kafkatest.version import TRUNK +from kafkatest.version import DEV_BRANCH class MiniKdc(KafkaPathResolverMixin, Service): @@ -103,8 +103,8 @@ def start_node(self, node): principals = 'client ' + kafka_principals + ' ' + self.extra_principals self.logger.info("Starting MiniKdc with principals " + principals) - core_libs_jar = self.path.jar(CORE_LIBS_JAR_NAME, TRUNK) - core_dependant_test_libs_jar = self.path.jar(CORE_DEPENDANT_TEST_LIBS_JAR_NAME, TRUNK) + core_libs_jar = self.path.jar(CORE_LIBS_JAR_NAME, DEV_BRANCH) + core_dependant_test_libs_jar = self.path.jar(CORE_DEPENDANT_TEST_LIBS_JAR_NAME, DEV_BRANCH) cmd = "for file in %s; do CLASSPATH=$CLASSPATH:$file; done;" % core_libs_jar cmd += " for file in %s; do CLASSPATH=$CLASSPATH:$file; done;" % core_dependant_test_libs_jar diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index 9b29217bf723e..beb37950ce139 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -94,6 +94,12 @@ class SecurityConfig(TemplateRenderer): SASL_SSL = 'SASL_SSL' SASL_MECHANISM_GSSAPI = 'GSSAPI' SASL_MECHANISM_PLAIN = 'PLAIN' + SASL_MECHANISM_SCRAM_SHA_256 = 'SCRAM-SHA-256' + SASL_MECHANISM_SCRAM_SHA_512 = 'SCRAM-SHA-512' + SCRAM_CLIENT_USER = "kafka-client" + SCRAM_CLIENT_PASSWORD = "client-secret" + SCRAM_BROKER_USER = "kafka-broker" + SCRAM_BROKER_PASSWORD = "broker-secret" CONFIG_DIR = "/mnt/security" KEYSTORE_PATH = "/mnt/security/test.keystore.jks" TRUSTSTORE_PATH = "/mnt/security/test.truststore.jks" @@ -106,7 +112,7 @@ class SecurityConfig(TemplateRenderer): def __init__(self, context, security_protocol=None, interbroker_security_protocol=None, client_sasl_mechanism=SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SASL_MECHANISM_GSSAPI, - zk_sasl=False, template_props=""): + zk_sasl=False, template_props="", static_jaas_conf=True): """ Initialize the security properties for the node and copy keystore and truststore to the remote node if the transport protocol @@ -137,6 +143,7 @@ def __init__(self, context, security_protocol=None, interbroker_security_protoco self.has_sasl = self.is_sasl(security_protocol) or self.is_sasl(interbroker_security_protocol) or zk_sasl self.has_ssl = self.is_ssl(security_protocol) or self.is_ssl(interbroker_security_protocol) self.zk_sasl = zk_sasl + self.static_jaas_conf = static_jaas_conf self.properties = { 'security.protocol' : security_protocol, 'ssl.keystore.location' : SecurityConfig.KEYSTORE_PATH, @@ -150,8 +157,18 @@ def __init__(self, context, security_protocol=None, interbroker_security_protoco 'sasl.kerberos.service.name' : 'kafka' } - def client_config(self, template_props=""): - return SecurityConfig(self.context, self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props) + def client_config(self, template_props="", node=None): + # If node is not specified, use static jaas config which will be created later. + # Otherwise use static JAAS configuration files with SASL_SSL and sasl.jaas.config + # property with SASL_PLAINTEXT so that both code paths are tested by existing tests. + # Note that this is an artibtrary choice and it is possible to run all tests with + # either static or dynamic jaas config files if required. + static_jaas_conf = node is None or (self.has_sasl and self.has_ssl) + return SecurityConfig(self.context, self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props, static_jaas_conf=static_jaas_conf) + + def enable_security_protocol(self, security_protocol): + self.has_sasl = self.has_sasl or self.is_sasl(security_protocol) + self.has_ssl = self.has_ssl or self.is_ssl(security_protocol) def setup_ssl(self, node): node.account.ssh("mkdir -p %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) @@ -167,9 +184,14 @@ def setup_sasl(self, node): else: is_ibm_jdk = False jaas_conf = self.render(jaas_conf_file, node=node, is_ibm_jdk=is_ibm_jdk, + SecurityConfig=SecurityConfig, client_sasl_mechanism=self.client_sasl_mechanism, - enabled_sasl_mechanisms=self.enabled_sasl_mechanisms) - node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) + enabled_sasl_mechanisms=self.enabled_sasl_mechanisms, + static_jaas_conf=self.static_jaas_conf) + if self.static_jaas_conf: + node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) + else: + self.properties['sasl.jaas.config'] = jaas_conf.replace("\n", " \\\n") if self.has_sasl_kerberos: node.account.copy_to(MiniKdc.LOCAL_KEYTAB_FILE, SecurityConfig.KEYTAB_PATH) node.account.copy_to(MiniKdc.LOCAL_KRB5CONF_FILE, SecurityConfig.KRB5CONF_PATH) @@ -181,6 +203,21 @@ def setup_node(self, node): if self.has_sasl: self.setup_sasl(node) + def setup_credentials(self, node, path, zk_connect, broker): + if broker: + self.maybe_create_scram_credentials(node, zk_connect, path, self.interbroker_sasl_mechanism, + SecurityConfig.SCRAM_BROKER_USER, SecurityConfig.SCRAM_BROKER_PASSWORD) + else: + self.maybe_create_scram_credentials(node, zk_connect, path, self.client_sasl_mechanism, + SecurityConfig.SCRAM_CLIENT_USER, SecurityConfig.SCRAM_CLIENT_PASSWORD) + + def maybe_create_scram_credentials(self, node, zk_connect, path, mechanism, user_name, password): + if self.has_sasl and self.is_sasl_scram(mechanism): + cmd = "%s --zookeeper %s --entity-name %s --entity-type users --alter --add-config %s=[password=%s]" % \ + (path.script("kafka-configs.sh", node), zk_connect, + user_name, mechanism, password) + node.account.ssh(cmd) + def clean_node(self, node): if self.security_protocol != SecurityConfig.PLAINTEXT: node.account.ssh("rm -rf %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) @@ -203,6 +240,9 @@ def is_ssl(self, security_protocol): def is_sasl(self, security_protocol): return security_protocol == SecurityConfig.SASL_PLAINTEXT or security_protocol == SecurityConfig.SASL_SSL + def is_sasl_scram(self, sasl_mechanism): + return sasl_mechanism == SecurityConfig.SASL_MECHANISM_SCRAM_SHA_256 or sasl_mechanism == SecurityConfig.SASL_MECHANISM_SCRAM_SHA_512 + @property def security_protocol(self): return self.properties['security.protocol'] @@ -226,7 +266,10 @@ def has_sasl_kerberos(self): @property def kafka_opts(self): if self.has_sasl: - return "\"-Djava.security.auth.login.config=%s -Djava.security.krb5.conf=%s\"" % (SecurityConfig.JAAS_CONF_PATH, SecurityConfig.KRB5CONF_PATH) + if self.static_jaas_conf: + return "\"-Djava.security.auth.login.config=%s -Djava.security.krb5.conf=%s\"" % (SecurityConfig.JAAS_CONF_PATH, SecurityConfig.KRB5CONF_PATH) + else: + return "\"-Djava.security.krb5.conf=%s\"" % SecurityConfig.KRB5CONF_PATH else: return "" @@ -240,6 +283,8 @@ def props(self, prefix=''): """ if self.security_protocol == SecurityConfig.PLAINTEXT: return "" + if self.has_sasl and not self.static_jaas_conf and 'sasl.jaas.config' not in self.properties: + raise Exception("JAAS configuration property has not yet been initialized") config_lines = (prefix + key + "=" + value for key, value in self.properties.iteritems()) # Extra blank lines ensure this can be appended/prepended safely return "\n".join(itertools.chain([""], config_lines, [""])) diff --git a/tests/kafkatest/services/security/templates/jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf index fbfa8af49c2db..e2511451e3272 100644 --- a/tests/kafkatest/services/security/templates/jaas.conf +++ b/tests/kafkatest/services/security/templates/jaas.conf @@ -12,7 +12,9 @@ */ +{% if static_jaas_conf %} KafkaClient { +{% endif %} {% if client_sasl_mechanism == "GSSAPI" %} {% if is_ibm_jdk %} com.ibm.security.auth.module.Krb5LoginModule required debug=false @@ -31,8 +33,13 @@ KafkaClient { org.apache.kafka.common.security.plain.PlainLoginModule required username="client" password="client-secret"; +{% elif client_sasl_mechanism == "SCRAM-SHA-256" or client_sasl_mechanism == "SCRAM-SHA-512" %} + org.apache.kafka.common.security.scram.ScramLoginModule required + username="{{ SecurityConfig.SCRAM_CLIENT_USER }}" + password="{{ SecurityConfig.SCRAM_CLIENT_PASSWORD }}"; {% endif %} +{% if static_jaas_conf %} }; KafkaServer { @@ -58,6 +65,11 @@ KafkaServer { user_client="client-secret" user_kafka="kafka-secret"; {% endif %} +{% if client_sasl_mechanism == "SCRAM-SHA-256" or client_sasl_mechanism == "SCRAM-SHA-512" %} + org.apache.kafka.common.security.scram.ScramLoginModule required + username="{{ SecurityConfig.SCRAM_BROKER_USER }}" + password="{{ SecurityConfig.SCRAM_BROKER_PASSWORD }}"; +{% endif %} }; {% if zk_sasl %} @@ -93,3 +105,4 @@ Server { {% endif %} }; {% endif %} +{% endif %} diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 87e441470e425..e7be9475f79f8 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -22,8 +22,8 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin -class StreamsSmokeTestBaseService(KafkaPathResolverMixin, Service): - """Base class for Streams Smoke Test services providing some common settings and functionality""" +class StreamsTestBaseService(KafkaPathResolverMixin, Service): + """Base class for Streams Test services providing some common settings and functionality""" PERSISTENT_ROOT = "/mnt/streams" # The log file contains normal log4j logs written using a file appender. stdout and stderr are handled separately @@ -45,10 +45,14 @@ class StreamsSmokeTestBaseService(KafkaPathResolverMixin, Service): "collect_default": True}, } - def __init__(self, context, kafka, command): - super(StreamsSmokeTestBaseService, self).__init__(context, 1) + def __init__(self, test_context, kafka, streams_class_name, user_test_args, user_test_args1=None, user_test_args2=None): + super(StreamsTestBaseService, self).__init__(test_context, 1) self.kafka = kafka - self.args = {'command': command} + self.args = {'streams_class_name': streams_class_name, + 'user_test_args': user_test_args, + 'user_test_args1': user_test_args1, + 'user_test_args2': user_test_args2} + self.log_level = "DEBUG" @property def node(self): @@ -65,7 +69,7 @@ def stop_nodes(self, clean_shutdown=True): self.stop_node(node, clean_shutdown) def stop_node(self, node, clean_shutdown=True): - self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping Streams Smoke Test on " + str(node.account)) + self.logger.info((clean_shutdown and "Cleanly" or "Forcibly") + " stopping Streams Test on " + str(node.account)) pids = self.pids(node) sig = signal.SIGTERM if clean_shutdown else signal.SIGKILL @@ -73,7 +77,7 @@ def stop_node(self, node, clean_shutdown=True): node.account.signal(pid, sig, allow_fail=True) if clean_shutdown: for pid in pids: - wait_until(lambda: not node.account.alive(pid), timeout_sec=60, err_msg="Streams Smoke Test process on " + str(node.account) + " took too long to exit") + wait_until(lambda: not node.account.alive(pid), timeout_sec=120, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") node.account.ssh("rm -f " + self.PID_FILE, allow_fail=False) @@ -93,10 +97,13 @@ def abortThenRestart(self): self.logger.info("Restarting Kafka Streams on " + str(node.account)) self.start_node(node) - def wait(self, timeout_sec=360): + def wait(self, timeout_sec=1440): for node in self.nodes: - for pid in self.pids(node): - wait_until(lambda: not node.account.alive(pid), timeout_sec=timeout_sec, err_msg="Streams Smoke Test process on " + str(node.account) + " took too long to exit") + self.wait_node(node, timeout_sec) + + def wait_node(self, node, timeout_sec=None): + for pid in self.pids(node): + wait_until(lambda: not node.account.alive(pid), timeout_sec=timeout_sec, err_msg="Streams Test process on " + str(node.account) + " took too long to exit") def clean_node(self, node): node.account.kill_process("streams", clean_shutdown=False, allow_fail=True) @@ -105,7 +112,6 @@ def clean_node(self, node): def start_cmd(self, node): args = self.args.copy() args['kafka'] = self.kafka.bootstrap_servers() - args['zk'] = self.kafka.zk.connect_setting() args['state_dir'] = self.PERSISTENT_ROOT args['stdout'] = self.STDOUT_FILE args['stderr'] = self.STDERR_FILE @@ -114,8 +120,8 @@ def start_cmd(self, node): args['kafka_run_class'] = self.path.script("kafka-run-class.sh", node) cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "INCLUDE_TEST_JARS=true %(kafka_run_class)s org.apache.kafka.streams.smoketest.StreamsSmokeTest " \ - " %(command)s %(kafka)s %(zk)s %(state_dir)s " \ + "INCLUDE_TEST_JARS=true %(kafka_run_class)s %(streams_class_name)s " \ + " %(kafka)s %(state_dir)s %(user_test_args)s %(user_test_args1)s %(user_test_args2)s" \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args return cmd @@ -125,24 +131,43 @@ def start_node(self, node): node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('tools_log4j.properties', log_file=self.LOG_FILE)) - self.logger.info("Starting StreamsSmokeTest process on " + str(node.account)) + self.logger.info("Starting StreamsTest process on " + str(node.account)) with node.account.monitor_log(self.STDOUT_FILE) as monitor: node.account.ssh(self.start_cmd(node)) - monitor.wait_until('StreamsSmokeTest instance started', timeout_sec=15, err_msg="Never saw message indicating StreamsSmokeTest finished startup on " + str(node.account)) + monitor.wait_until('StreamsTest instance started', timeout_sec=15, err_msg="Never saw message indicating StreamsTest finished startup on " + str(node.account)) if len(self.pids(node)) == 0: raise RuntimeError("No process ids recorded") +class StreamsSmokeTestBaseService(StreamsTestBaseService): + """Base class for Streams Smoke Test services providing some common settings and functionality""" + + def __init__(self, test_context, kafka, command): + super(StreamsSmokeTestBaseService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.tests.StreamsSmokeTest", + command) + + class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestDriverService, self).__init__(context, kafka, "run") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestDriverService, self).__init__(test_context, kafka, "run") class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestJobRunnerService, self).__init__(context, kafka, "process") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestJobRunnerService, self).__init__(test_context, kafka, "process") + class StreamsSmokeTestShutdownDeadlockService(StreamsSmokeTestBaseService): - def __init__(self, context, kafka): - super(StreamsSmokeTestShutdownDeadlockService, self).__init__(context, kafka, "close-deadlock-test") + def __init__(self, test_context, kafka): + super(StreamsSmokeTestShutdownDeadlockService, self).__init__(test_context, kafka, "close-deadlock-test") + + +class StreamsBrokerCompatibilityService(StreamsTestBaseService): + def __init__(self, test_context, kafka): + super(StreamsBrokerCompatibilityService, self).__init__(test_context, + kafka, + "org.apache.kafka.streams.tests.BrokerCompatibilityTest", + "dummy") diff --git a/tests/kafkatest/services/verifiable_consumer.py b/tests/kafkatest/services/verifiable_consumer.py index c593e2a79bd42..013913239e8ba 100644 --- a/tests/kafkatest/services/verifiable_consumer.py +++ b/tests/kafkatest/services/verifiable_consumer.py @@ -22,7 +22,7 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.services.kafka import TopicPartition -from kafkatest.version import TRUNK +from kafkatest.version import DEV_BRANCH class ConsumerState: @@ -55,12 +55,17 @@ def handle_offsets_committed(self, event): partition = offset_commit["partition"] tp = TopicPartition(topic, partition) offset = offset_commit["offset"] - assert tp in self.assignment, "Committed offsets for a partition not assigned" - assert self.position[tp] >= offset, "The committed offset was greater than the current position" + assert tp in self.assignment, \ + "Committed offsets for partition %s not assigned (current assignment: %s)" % \ + (str(tp), str(self.assignment)) + assert self.position[tp] >= offset, \ + "The committed offset %d was greater than the current position %d for partition %s" % \ + (offset, self.position[t], str(tp)) self.committed[tp] = offset def handle_records_consumed(self, event): - assert self.state == ConsumerState.Joined, "Consumed records should only be received when joined" + assert self.state == ConsumerState.Joined, \ + "Consumed records should only be received when joined (current state: %s)" % str(self.state) for record_batch in event["partitions"]: tp = TopicPartition(topic=record_batch["topic"], @@ -68,9 +73,12 @@ def handle_records_consumed(self, event): min_offset = record_batch["minOffset"] max_offset = record_batch["maxOffset"] - assert tp in self.assignment, "Consumed records for a partition not assigned" + assert tp in self.assignment, \ + "Consumed records for partition %s which is not assigned (current assignment: %s)" % \ + (str(tp), str(self.assignment)) assert tp not in self.position or self.position[tp] == min_offset, \ - "Consumed from an unexpected offset (%s, %s)" % (str(self.position[tp]), str(min_offset)) + "Consumed from an unexpected offset (%d, %d) for partition %s" % \ + (self.position[tp], min_offset, str(tp)) self.position[tp] = max_offset + 1 self.total_consumed += event["count"] @@ -136,9 +144,9 @@ class VerifiableConsumer(KafkaPathResolverMixin, BackgroundThreadService): def __init__(self, context, num_nodes, kafka, topic, group_id, max_messages=-1, session_timeout_sec=30, enable_autocommit=False, assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor", - version=TRUNK, stop_timeout_sec=30): + version=DEV_BRANCH, stop_timeout_sec=30, log_level="INFO"): super(VerifiableConsumer, self).__init__(context, num_nodes) - self.log_level = "TRACE" + self.log_level = log_level self.kafka = kafka self.topic = topic @@ -148,8 +156,6 @@ def __init__(self, context, num_nodes, kafka, topic, group_id, self.enable_autocommit = enable_autocommit self.assignment_strategy = assignment_strategy self.prop_file = "" - self.security_config = kafka.security_config.client_config(self.prop_file) - self.prop_file += str(self.security_config) self.stop_timeout_sec = stop_timeout_sec self.event_handlers = {} @@ -160,10 +166,11 @@ def __init__(self, context, num_nodes, kafka, topic, group_id, node.version = version def _worker(self, idx, node): - if node not in self.event_handlers: - self.event_handlers[node] = ConsumerEventHandler(node) + with self.lock: + if node not in self.event_handlers: + self.event_handlers[node] = ConsumerEventHandler(node) + handler = self.event_handlers[node] - handler = self.event_handlers[node] node.account.ssh("mkdir -p %s" % VerifiableConsumer.PERSISTENT_ROOT, allow_fail=False) # Create and upload log properties @@ -171,6 +178,9 @@ def _worker(self, idx, node): node.account.create_file(VerifiableConsumer.LOG4J_CONFIG, log_config) # Create and upload config file + self.security_config = self.kafka.security_config.client_config(self.prop_file, node) + self.security_config.setup_node(node) + self.prop_file += str(self.security_config) self.logger.info("verifiable_consumer.properties:") self.logger.info(self.prop_file) node.account.create_file(VerifiableConsumer.CONFIG_FILE, self.prop_file) @@ -202,13 +212,14 @@ def _update_global_position(self, consumed_event): if tp in self.global_committed: # verify that the position never gets behind the current commit. assert self.global_committed[tp] <= consumed_partition["minOffset"], \ - "Consumed position %d is behind the current committed offset %d" % (consumed_partition["minOffset"], self.global_committed[tp]) + "Consumed position %d is behind the current committed offset %d for partition %s" % \ + (consumed_partition["minOffset"], self.global_committed[tp], str(tp)) # the consumer cannot generally guarantee that the position increases monotonically # without gaps in the face of hard failures, so we only log a warning when this happens if tp in self.global_position and self.global_position[tp] != consumed_partition["minOffset"]: - self.logger.warn("Expected next consumed offset of %d, but instead saw %d" % - (self.global_position[tp], consumed_partition["minOffset"])) + self.logger.warn("Expected next consumed offset of %d for partition %s, but instead saw %d" % + (self.global_position[tp], str(tp), consumed_partition["minOffset"])) self.global_position[tp] = consumed_partition["maxOffset"] + 1 @@ -218,7 +229,8 @@ def _update_global_committed(self, commit_event): tp = TopicPartition(offset_commit["topic"], offset_commit["partition"]) offset = offset_commit["offset"] assert self.global_position[tp] >= offset, \ - "committed offset is ahead of the current partition" + "Committed offset %d for partition %s is ahead of the current position %d" % \ + (offset, str(tp), self.global_position[tp]) self.global_committed[tp] = offset def start_cmd(self, node): @@ -266,7 +278,8 @@ def kill_node(self, node, clean_shutdown=True, allow_fail=False): for pid in self.pids(node): node.account.signal(pid, sig, allow_fail) - self.event_handlers[node].handle_kill_process(clean_shutdown) + with self.lock: + self.event_handlers[node].handle_kill_process(clean_shutdown) def stop_node(self, node, clean_shutdown=True): self.kill_node(node, clean_shutdown=clean_shutdown) @@ -292,10 +305,11 @@ def current_position(self, tp): return None def owner(self, tp): - for handler in self.event_handlers.itervalues(): - if tp in handler.current_assignment(): - return handler.node - return None + with self.lock: + for handler in self.event_handlers.itervalues(): + if tp in handler.current_assignment(): + return handler.node + return None def last_commit(self, tp): with self.lock: diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index 205143e411839..5f39df21f9339 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -24,7 +24,7 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin, TOOLS_JAR_NAME, TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME from kafkatest.utils import is_int, is_int_with_prefix -from kafkatest.version import TRUNK, LATEST_0_8_2 +from kafkatest.version import DEV_BRANCH, LATEST_0_8_2 from kafkatest.utils.remote_account import line_count @@ -50,8 +50,8 @@ class VerifiableProducer(KafkaPathResolverMixin, BackgroundThreadService): } def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000, - message_validator=is_int, compression_types=None, version=TRUNK, acks=None, - stop_timeout_sec=150): + message_validator=is_int, compression_types=None, version=DEV_BRANCH, acks=None, + stop_timeout_sec=150, request_timeout_sec=30, log_level="INFO"): """ :param max_messages is a number of messages to be produced per producer :param message_validator checks for an expected format of messages produced. There are @@ -64,6 +64,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput compression types, one per producer (could be "none"). """ super(VerifiableProducer, self).__init__(context, num_nodes) + self.log_level = log_level self.kafka = kafka self.topic = topic @@ -82,10 +83,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput self.clean_shutdown_nodes = set() self.acks = acks self.stop_timeout_sec = stop_timeout_sec - - @property - def security_config(self): - return self.kafka.security_config.client_config() + self.request_timeout_sec = request_timeout_sec def prop_file(self, node): idx = self.idx(node) @@ -104,15 +102,20 @@ def _worker(self, idx, node): log_config = self.render('tools_log4j.properties', log_file=VerifiableProducer.LOG_FILE) node.account.create_file(VerifiableProducer.LOG4J_CONFIG, log_config) + # Configure security + self.security_config = self.kafka.security_config.client_config(node=node) + self.security_config.setup_node(node) + # Create and upload config file producer_prop_file = self.prop_file(node) if self.acks is not None: self.logger.info("VerifiableProducer (index = %d) will use acks = %s", idx, self.acks) producer_prop_file += "\nacks=%s\n" % self.acks + + producer_prop_file += "\nrequest.timeout.ms=%d\n" % (self.request_timeout_sec * 1000) self.logger.info("verifiable_producer.properties:") self.logger.info(producer_prop_file) node.account.create_file(VerifiableProducer.CONFIG_FILE, producer_prop_file) - self.security_config.setup_node(node) cmd = self.start_cmd(node, idx) self.logger.debug("VerifiableProducer %d command: %s" % (idx, cmd)) @@ -182,9 +185,9 @@ def start_cmd(self, node, idx): cmd = "" if node.version <= LATEST_0_8_2: # 0.8.2.X releases do not have VerifiableProducer.java, so cheat and add - # the tools jar from trunk to the classpath - tools_jar = self.path.jar(TOOLS_JAR_NAME, TRUNK) - tools_dependant_libs_jar = self.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, TRUNK) + # the tools jar from the development branch to the classpath + tools_jar = self.path.jar(TOOLS_JAR_NAME, DEV_BRANCH) + tools_dependant_libs_jar = self.path.jar(TOOLS_DEPENDANT_TEST_LIBS_JAR_NAME, DEV_BRANCH) cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_jar cmd += "for file in %s; do CLASSPATH=$CLASSPATH:$file; done; " % tools_dependant_libs_jar diff --git a/tests/kafkatest/services/zookeeper.py b/tests/kafkatest/services/zookeeper.py index 8d38d48535143..571956b1a6a45 100644 --- a/tests/kafkatest/services/zookeeper.py +++ b/tests/kafkatest/services/zookeeper.py @@ -23,7 +23,7 @@ from kafkatest.directory_layout.kafka_path import KafkaPathResolverMixin from kafkatest.services.security.security_config import SecurityConfig -from kafkatest.version import TRUNK +from kafkatest.version import DEV_BRANCH class ZookeeperService(KafkaPathResolverMixin, Service): @@ -122,7 +122,7 @@ def query(self, path): """ Queries zookeeper for data associated with 'path' and returns all fields in the schema """ - kafka_run_class = self.path.script("kafka-run-class.sh", TRUNK) + kafka_run_class = self.path.script("kafka-run-class.sh", DEV_BRANCH) cmd = "%s kafka.tools.ZooKeeperMainWrapper -server %s get %s" % \ (kafka_run_class, self.connect_setting(), path) self.logger.debug(cmd) diff --git a/tests/kafkatest/tests/client/client_compatibility_features_test.py b/tests/kafkatest/tests/client/client_compatibility_features_test.py new file mode 100644 index 0000000000000..d478590dcfd36 --- /dev/null +++ b/tests/kafkatest/tests/client/client_compatibility_features_test.py @@ -0,0 +1,104 @@ +# 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. + +import os +import time +from random import randint + +from ducktape.mark import parametrize +from ducktape.tests.test import TestContext + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from ducktape.tests.test import Test +from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, V_0_10_1_0, KafkaVersion + +def get_broker_features(broker_version): + features = {} + if (broker_version < V_0_10_1_0): + features["offsets-for-times-supported"] = False + features["cluster-id-supported"] = False + features["expect-record-too-large-exception"] = True + else: + features["offsets-for-times-supported"] = True + features["cluster-id-supported"] = True + features["expect-record-too-large-exception"] = False + return features + +def run_command(node, cmd, ssh_log_file): + with open(ssh_log_file, 'w') as f: + f.write("Running %s\n" % cmd) + try: + for line in node.account.ssh_capture(cmd): + f.write(line) + except Exception as e: + f.write("** Command failed!") + print e + raise e + + +class ClientCompatibilityFeaturesTest(Test): + """ + Tests clients for the presence or absence of specific features when communicating with brokers with various + versions. Relies on ClientCompatibilityTest.java for much of the functionality. + """ + + def __init__(self, test_context): + """:type test_context: ducktape.tests.test.TestContext""" + super(ClientCompatibilityFeaturesTest, self).__init__(test_context=test_context) + + self.zk = ZookeeperService(test_context, num_nodes=3) + + # Generate a unique topic name + topic_name = "client_compat_features_topic_%d%d" % (int(time.time()), randint(0, 2147483647)) + self.topics = { topic_name: { + "partitions": 1, # Use only one partition to avoid worrying about ordering + "replication-factor": 3 + }} + self.kafka = KafkaService(test_context, num_nodes=3, zk=self.zk, topics=self.topics) + + def invoke_compatibility_program(self, features): + # Run the compatibility test on the first Kafka node. + node = self.zk.nodes[0] + cmd = ("%s org.apache.kafka.tools.ClientCompatibilityTest " + "--bootstrap-server %s " + "--offsets-for-times-supported %s " + "--cluster-id-supported %s " + "--expect-record-too-large-exception %s " + "--topic %s " % (self.zk.path.script("kafka-run-class.sh", node), + self.kafka.bootstrap_servers(), + features["offsets-for-times-supported"], + features["cluster-id-supported"], + features["expect-record-too-large-exception"], + self.topics.keys()[0])) + results_dir = TestContext.results_dir(self.test_context, 0) + os.makedirs(results_dir) + ssh_log_file = "%s/%s" % (results_dir, "client_compatibility_test_output.txt") + try: + self.logger.info("Running %s" % cmd) + run_command(node, cmd, ssh_log_file) + except Exception as e: + self.logger.info("** Command failed. See %s for log messages." % ssh_log_file) + raise e + + @parametrize(broker_version=str(DEV_BRANCH)) + @parametrize(broker_version=str(LATEST_0_10_0)) + @parametrize(broker_version=str(LATEST_0_10_1)) + def run_compatibility_test(self, broker_version): + self.zk.start() + self.kafka.set_version(KafkaVersion(broker_version)) + self.kafka.start() + features = get_broker_features(broker_version) + self.invoke_compatibility_program(features) diff --git a/tests/kafkatest/tests/client/test_producer_consumer_compat.py b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py similarity index 83% rename from tests/kafkatest/tests/client/test_producer_consumer_compat.py rename to tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py index 5a14fc5995cf4..c5e80cbec4d0a 100644 --- a/tests/kafkatest/tests/client/test_producer_consumer_compat.py +++ b/tests/kafkatest/tests/client/client_compatibility_produce_consume_test.py @@ -22,17 +22,16 @@ from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int_with_prefix -from kafkatest.version import TRUNK, V_0_10_0_0, KafkaVersion +from kafkatest.version import DEV_BRANCH, LATEST_0_10_0, LATEST_0_10_1, KafkaVersion -class TestProducerConsumerCompat(ProduceConsumeValidateTest): +class ClientCompatibilityProduceConsumeTest(ProduceConsumeValidateTest): """ - These tests validate that we can use a new client to consume from older - brokers. + These tests validate that we can use a new client to produce and consume from older brokers. """ def __init__(self, test_context): """:type test_context: ducktape.tests.test.TestContext""" - super(TestProducerConsumerCompat, self).__init__(test_context=test_context) + super(ClientCompatibilityProduceConsumeTest, self).__init__(test_context=test_context) self.topic = "test_topic" self.zk = ZookeeperService(test_context, num_nodes=3) @@ -51,10 +50,11 @@ def setUp(self): def min_cluster_size(self): # Override this since we're adding services outside of the constructor - return super(TestProducerConsumerCompat, self).min_cluster_size() + self.num_producers + self.num_consumers + return super(ClientCompatibilityProduceConsumeTest, self).min_cluster_size() + self.num_producers + self.num_consumers - # TODO: when KAFKA-4462 is fully implemented, we should test other versions here. - @parametrize(broker_version=str(TRUNK)) + @parametrize(broker_version=str(DEV_BRANCH)) + @parametrize(broker_version=str(LATEST_0_10_0)) + @parametrize(broker_version=str(LATEST_0_10_1)) def test_produce_consume(self, broker_version): print("running producer_consumer_compat with broker_version = %s" % broker_version) self.kafka.set_version(KafkaVersion(broker_version)) diff --git a/tests/kafkatest/tests/client/consumer_test.py b/tests/kafkatest/tests/client/consumer_test.py index a68e23ec803af..9eb5ab6890e6b 100644 --- a/tests/kafkatest/tests/client/consumer_test.py +++ b/tests/kafkatest/tests/client/consumer_test.py @@ -116,7 +116,7 @@ def test_broker_rolling_bounce(self): "Total consumed records did not match consumed position" @cluster(num_nodes=7) - @matrix(clean_shutdown=[True, False], bounce_mode=["all", "rolling"]) + @matrix(clean_shutdown=[True], bounce_mode=["all", "rolling"]) def test_consumer_bounce(self, clean_shutdown, bounce_mode): """ Verify correct consumer behavior when the consumers in the group are consecutively restarted. @@ -150,14 +150,16 @@ def test_consumer_bounce(self, clean_shutdown, bounce_mode): # if the total records consumed matches the current position, we haven't seen any duplicates # this can only be guaranteed with a clean shutdown assert consumer.current_position(partition) == consumer.total_consumed(), \ - "Total consumed records did not match consumed position" + "Total consumed records %d did not match consumed position %d" % \ + (consumer.total_consumed(), consumer.current_position(partition)) else: # we may have duplicates in a hard failure assert consumer.current_position(partition) <= consumer.total_consumed(), \ - "Current position greater than the total number of consumed records" + "Current position %d greater than the total number of consumed records %d" % \ + (consumer.current_position(partition), consumer.total_consumed()) @cluster(num_nodes=7) - @matrix(clean_shutdown=[True, False], enable_autocommit=[True, False]) + @matrix(clean_shutdown=[True], enable_autocommit=[True, False]) def test_consumer_failure(self, clean_shutdown, enable_autocommit): partition = TopicPartition(self.TOPIC, 0) diff --git a/tests/kafkatest/tests/client/message_format_change_test.py b/tests/kafkatest/tests/client/message_format_change_test.py index edcead2abf080..e58f58b06b156 100644 --- a/tests/kafkatest/tests/client/message_format_change_test.py +++ b/tests/kafkatest/tests/client/message_format_change_test.py @@ -22,7 +22,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import LATEST_0_9, LATEST_0_10, TRUNK, KafkaVersion +from kafkatest.version import LATEST_0_9, LATEST_0_10, DEV_BRANCH, KafkaVersion class MessageFormatChangeTest(ProduceConsumeValidateTest): @@ -58,7 +58,7 @@ def produce_and_consume(self, producer_version, consumer_version, group): err_msg="Producer did not produce all messages in reasonable amount of time")) @cluster(num_nodes=10) - @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK)) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH)) @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(LATEST_0_9)) def test_compatibility(self, producer_version, consumer_version): """ This tests performs the following checks: @@ -70,7 +70,7 @@ def test_compatibility(self, producer_version, consumer_version): - The producers and consumers should not have any issue. - Note that for 0.9.x consumers/producers we only do steps 1 and 2 """ - self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=TRUNK, topics={self.topic: { + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=DEV_BRANCH, topics={self.topic: { "partitions": 3, "replication-factor": 3, 'configs': {"min.insync.replicas": 2}}}) @@ -84,7 +84,7 @@ def test_compatibility(self, producer_version, consumer_version): self.kafka.alter_message_format(self.topic, str(LATEST_0_10)) self.produce_and_consume(producer_version, consumer_version, "group2") - if producer_version == str(TRUNK) and consumer_version == str(TRUNK): + if producer_version == str(DEV_BRANCH) and consumer_version == str(DEV_BRANCH): self.logger.info("Third format change back to 0.9.0") self.kafka.alter_message_format(self.topic, str(LATEST_0_9)) self.produce_and_consume(producer_version, consumer_version, "group3") diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index 198e94501f50d..c298fb154e86e 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -460,7 +460,7 @@ def test_transformations(self): 'file': self.INPUT_FILE, 'topic': self.TOPIC, 'transforms': 'hoistToStruct,insertTimestampField', - 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistToStruct$Value', + 'transforms.hoistToStruct.type': 'org.apache.kafka.connect.transforms.HoistField$Value', 'transforms.hoistToStruct.field': 'content', 'transforms.insertTimestampField.type': 'org.apache.kafka.connect.transforms.InsertField$Value', 'transforms.insertTimestampField.timestamp.field': ts_fieldname, diff --git a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py index f3931ec9864f3..da6483d9023b3 100644 --- a/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py +++ b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py @@ -23,7 +23,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import LATEST_0_10_0, LATEST_0_9, LATEST_0_8_2, TRUNK, KafkaVersion +from kafkatest.version import LATEST_0_10_1, LATEST_0_10_0, LATEST_0_9, LATEST_0_8_2, DEV_BRANCH, KafkaVersion # Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x) @@ -47,17 +47,18 @@ def setUp(self): @cluster(num_nodes=6) @parametrize(producer_version=str(LATEST_0_8_2), consumer_version=str(LATEST_0_8_2), compression_types=["none"], new_consumer=False, timestamp_type=None) @parametrize(producer_version=str(LATEST_0_8_2), consumer_version=str(LATEST_0_9), compression_types=["none"], new_consumer=False, timestamp_type=None) - @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(TRUNK), compression_types=["none"], new_consumer=False, timestamp_type=None) - @parametrize(producer_version=str(TRUNK), consumer_version=str(LATEST_0_9), compression_types=["none"], new_consumer=False, timestamp_type=None) - @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(TRUNK), compression_types=["snappy"], timestamp_type=None) - @parametrize(producer_version=str(TRUNK), consumer_version=str(LATEST_0_9), compression_types=["snappy"], timestamp_type=str("CreateTime")) - @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) + @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(DEV_BRANCH), compression_types=["none"], new_consumer=False, timestamp_type=None) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(LATEST_0_9), compression_types=["none"], new_consumer=False, timestamp_type=None) + @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(DEV_BRANCH), compression_types=["snappy"], timestamp_type=None) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(LATEST_0_9), compression_types=["snappy"], timestamp_type=str("CreateTime")) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) + @parametrize(producer_version=str(LATEST_0_10_1), consumer_version=str(LATEST_0_10_1), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) @parametrize(producer_version=str(LATEST_0_10_0), consumer_version=str(LATEST_0_10_0), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(LATEST_0_9), compression_types=["snappy"], timestamp_type=str("LogAppendTime")) - @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK), compression_types=["none"], new_consumer=False, timestamp_type=str("LogAppendTime")) + @parametrize(producer_version=str(DEV_BRANCH), consumer_version=str(DEV_BRANCH), compression_types=["none"], new_consumer=False, timestamp_type=str("LogAppendTime")) def test_compatibility(self, producer_version, consumer_version, compression_types, new_consumer=True, timestamp_type=None): - self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=TRUNK, topics={self.topic: { + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=DEV_BRANCH, topics={self.topic: { "partitions": 3, "replication-factor": 3, 'configs': {"min.insync.replicas": 2}}}) diff --git a/tests/kafkatest/tests/core/replication_test.py b/tests/kafkatest/tests/core/replication_test.py index a95e9e5c289b9..3e17d5636eca3 100644 --- a/tests/kafkatest/tests/core/replication_test.py +++ b/tests/kafkatest/tests/core/replication_test.py @@ -16,6 +16,7 @@ from ducktape.utils.util import wait_until from ducktape.mark import matrix +from ducktape.mark import parametrize from ducktape.mark.resource import cluster from kafkatest.services.zookeeper import ZookeeperService @@ -122,13 +123,16 @@ def min_cluster_size(self): @cluster(num_nodes=7) @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"], broker_type=["leader"], - security_protocol=["PLAINTEXT", "SSL", "SASL_PLAINTEXT", "SASL_SSL"]) + security_protocol=["PLAINTEXT", "SASL_SSL"]) @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"], broker_type=["controller"], security_protocol=["PLAINTEXT", "SASL_SSL"]) @matrix(failure_mode=["hard_bounce"], broker_type=["leader"], security_protocol=["SASL_SSL"], client_sasl_mechanism=["PLAIN"], interbroker_sasl_mechanism=["PLAIN", "GSSAPI"]) + @parametrize(failure_mode="hard_bounce", + broker_type="leader", + security_protocol="SASL_SSL", client_sasl_mechanism="SCRAM-SHA-256", interbroker_sasl_mechanism="SCRAM-SHA-512") def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type, client_sasl_mechanism="GSSAPI", interbroker_sasl_mechanism="GSSAPI"): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by diff --git a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py index a21e845649d71..a962a9d140222 100644 --- a/tests/kafkatest/tests/core/security_rolling_upgrade_test.py +++ b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py @@ -69,8 +69,6 @@ def roll_in_secured_settings(self, client_protocol, broker_protocol): # Roll cluster to include inter broker security protocol. self.kafka.interbroker_security_protocol = broker_protocol - self.kafka.open_port(client_protocol) - self.kafka.open_port(broker_protocol) self.bounce() # Roll cluster to disable PLAINTEXT port @@ -131,14 +129,16 @@ def test_rolling_upgrade_phase_one(self, client_protocol): def test_rolling_upgrade_phase_two(self, client_protocol, broker_protocol): """ Start with a PLAINTEXT cluster with a second Secured port open (i.e. result of phase one). - Start an Producer and Consumer via the SECURED port - Incrementally upgrade to add inter-broker be the secure protocol + A third secure port is also open if inter-broker and client protocols are different. + Start a Producer and Consumer via the SECURED client port + Incrementally upgrade to add inter-broker be the secure broker protocol Incrementally upgrade again to add ACLs as well as disabling the PLAINTEXT port Ensure the producer and consumer ran throughout """ #Given we have a broker that has both secure and PLAINTEXT ports open self.kafka.security_protocol = client_protocol self.kafka.interbroker_security_protocol = "PLAINTEXT" + self.kafka.open_port(broker_protocol) self.kafka.start() #Create Secured Producer and Consumer diff --git a/tests/kafkatest/tests/core/throttling_test.py b/tests/kafkatest/tests/core/throttling_test.py index 80b5658ac3029..94a40106d0a08 100644 --- a/tests/kafkatest/tests/core/throttling_test.py +++ b/tests/kafkatest/tests/core/throttling_test.py @@ -15,7 +15,7 @@ import time import math -from ducktape.mark import parametrize,ignore +from ducktape.mark import parametrize from ducktape.mark.resource import cluster from ducktape.utils.util import wait_until @@ -53,7 +53,7 @@ def __init__(self, test_context): # ensure that the consumer is fully started before the producer starts # so that we don't miss any messages. This timeout ensures the sufficient # condition. - self.consumer_init_timeout_sec = 10 + self.consumer_init_timeout_sec = 20 self.num_brokers = 6 self.num_partitions = 3 self.kafka = KafkaService(test_context, @@ -133,15 +133,14 @@ def reassign_partitions(self, bounce_brokers, throttle): self.logger.debug("Transfer took %d second. Estimated time : %ds", time_taken, estimated_throttled_time) - assert time_taken >= estimated_throttled_time, \ + assert time_taken >= estimated_throttled_time * 0.9, \ ("Expected rebalance to take at least %ds, but it took %ds" % ( estimated_throttled_time, time_taken)) - @ignore @cluster(num_nodes=10) - @parametrize(bounce_brokers=False) @parametrize(bounce_brokers=True) + @parametrize(bounce_brokers=False) def test_throttled_reassignment(self, bounce_brokers): security_protocol = 'PLAINTEXT' self.kafka.security_protocol = security_protocol diff --git a/tests/kafkatest/tests/core/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py index 34af4eb56c7e1..8d89b7c13c36d 100644 --- a/tests/kafkatest/tests/core/upgrade_test.py +++ b/tests/kafkatest/tests/core/upgrade_test.py @@ -25,7 +25,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest from kafkatest.utils import is_int -from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, TRUNK, KafkaVersion +from kafkatest.version import LATEST_0_8_2, LATEST_0_9, LATEST_0_10_0, LATEST_0_10_1, DEV_BRANCH, KafkaVersion class TestUpgrade(ProduceConsumeValidateTest): @@ -46,7 +46,7 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.logger.info("First pass bounce - rolling upgrade") for node in self.kafka.nodes: self.kafka.stop_node(node) - node.version = TRUNK + node.version = DEV_BRANCH node.config[config_property.INTER_BROKER_PROTOCOL_VERSION] = from_kafka_version node.config[config_property.MESSAGE_FORMAT_VERSION] = from_kafka_version self.kafka.start_node(node) @@ -62,6 +62,8 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.kafka.start_node(node) @cluster(num_nodes=6) + @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["lz4"]) + @parametrize(from_kafka_version=str(LATEST_0_10_1), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) @parametrize(from_kafka_version=str(LATEST_0_10_0), to_message_format_version=None, compression_types=["snappy"]) @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["none"], new_consumer=False) @@ -80,7 +82,7 @@ def perform_upgrade(self, from_kafka_version, to_message_format_version=None): @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["snappy"], new_consumer=False) def test_upgrade(self, from_kafka_version, to_message_format_version, compression_types, new_consumer=True, security_protocol="PLAINTEXT"): - """Test upgrade of Kafka broker cluster from 0.8.2, 0.9.0 or 0.10.0 to the current version + """Test upgrade of Kafka broker cluster from 0.8.2, 0.9.0, 0.10.0, 0.10.1 to the current version from_kafka_version is a Kafka version to upgrade from: either 0.8.2.X, 0.9.0.x or 0.10.0.x @@ -112,7 +114,8 @@ def test_upgrade(self, from_kafka_version, to_message_format_version, compressio compression_types=compression_types, version=KafkaVersion(from_kafka_version)) - assert self.zk.query("/cluster/id") is None + if from_kafka_version <= LATEST_0_10_0: + assert self.zk.query("/cluster/id") is None # TODO - reduce the timeout self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, diff --git a/tests/kafkatest/tests/produce_consume_validate.py b/tests/kafkatest/tests/produce_consume_validate.py index 801ccde2e4e94..cad9150fcd0a0 100644 --- a/tests/kafkatest/tests/produce_consume_validate.py +++ b/tests/kafkatest/tests/produce_consume_validate.py @@ -15,7 +15,7 @@ from ducktape.tests.test import Test from ducktape.utils.util import wait_until - +import time class ProduceConsumeValidateTest(Test): """This class provides a shared template for tests which follow the common pattern of: @@ -50,12 +50,28 @@ def start_producer_and_consumer(self): # Start background producer and consumer self.consumer.start() if (self.consumer_init_timeout_sec > 0): - self.logger.debug("Waiting %ds for the consumer to fork.", + self.logger.debug("Waiting %ds for the consumer to initialize.", self.consumer_init_timeout_sec) + start = int(time.time()) wait_until(lambda: self.consumer.alive(self.consumer.nodes[0]) is True, timeout_sec=self.consumer_init_timeout_sec, err_msg="Consumer process took more than %d s to fork" %\ self.consumer_init_timeout_sec) + end = int(time.time()) + # If `JMXConnectFactory.connect` is invoked during the + # initialization of the JMX server, it may fail to throw the + # specified IOException back to the calling code. The sleep is a + # workaround that should allow initialization to complete before we + # try to connect. See KAFKA-4620 for more details. + time.sleep(1) + remaining_time = self.consumer_init_timeout_sec - (end - start) + if remaining_time < 0 : + remaining_time = 0 + if self.consumer.new_consumer is True: + wait_until(lambda: self.consumer.has_partitions_assigned(self.consumer.nodes[0]) is True, + timeout_sec=remaining_time, + err_msg="Consumer process took more than %d s to have partitions assigned" %\ + remaining_time) self.producer.start() wait_until(lambda: self.producer.num_acked > 5, diff --git a/tests/kafkatest/tests/streams/streams_bounce_test.py b/tests/kafkatest/tests/streams/streams_bounce_test.py index 169bbc1aa02dc..7ac79396381b0 100644 --- a/tests/kafkatest/tests/streams/streams_bounce_test.py +++ b/tests/kafkatest/tests/streams/streams_bounce_test.py @@ -26,7 +26,7 @@ class StreamsBounceTest(KafkaTest): """ def __init__(self, test_context): - super(StreamsBounceTest, self).__init__(test_context, num_zk=1, num_brokers=2, topics={ + super(StreamsBounceTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ 'echo' : { 'partitions': 5, 'replication-factor': 2 }, 'data' : { 'partitions': 5, 'replication-factor': 2 }, 'min' : { 'partitions': 5, 'replication-factor': 2 }, @@ -42,7 +42,7 @@ def __init__(self, test_context): self.driver = StreamsSmokeTestDriverService(test_context, self.kafka) self.processor1 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) - @cluster(num_nodes=5) + @cluster(num_nodes=6) def test_bounce(self): """ Start a smoke test client, then abort (kill -9) and restart it a few times. diff --git a/tests/kafkatest/tests/streams/streams_broker_bounce_test.py b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py new file mode 100644 index 0000000000000..86c19f9017cd5 --- /dev/null +++ b/tests/kafkatest/tests/streams/streams_broker_bounce_test.py @@ -0,0 +1,213 @@ +# 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. + +from ducktape.tests.test import Test +from ducktape.mark.resource import cluster +from ducktape.mark import matrix +from ducktape.mark import parametrize, ignore +from kafkatest.services.kafka import KafkaService +from kafkatest.tests.kafka_test import KafkaTest +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService +import time +import signal +from random import randint + +def broker_node(test, topic, broker_type): + """ Discover node of requested type. For leader type, discovers leader for our topic and partition 0 + """ + if broker_type == "leader": + node = test.kafka.leader(topic, partition=0) + elif broker_type == "controller": + node = test.kafka.controller() + else: + raise Exception("Unexpected broker type %s." % (broker_type)) + + return node + +def signal_node(test, node, sig): + test.kafka.signal_node(node, sig) + +def clean_shutdown(test, topic, broker_type): + """Discover broker node of requested type and shut it down cleanly. + """ + node = broker_node(test, topic, broker_type) + signal_node(test, node, signal.SIGTERM) + +def hard_shutdown(test, topic, broker_type): + """Discover broker node of requested type and shut it down with a hard kill.""" + node = broker_node(test, topic, broker_type) + signal_node(test, node, signal.SIGKILL) + + +failures = { + "clean_shutdown": clean_shutdown, + "hard_shutdown": hard_shutdown +} + +class StreamsBrokerBounceTest(Test): + """ + Simple test of Kafka Streams with brokers failing + """ + + def __init__(self, test_context): + super(StreamsBrokerBounceTest, self).__init__(test_context) + self.replication = 3 + self.partitions = 3 + self.topics = { + 'echo' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2}}, + 'data' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'min' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'max' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'sum' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'dif' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'cnt' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'avg' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'wcnt' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} }, + 'tagg' : { 'partitions': self.partitions, 'replication-factor': self.replication, + 'configs': {"min.insync.replicas": 2} } + } + + def fail_broker_type(self, failure_mode, broker_type): + # Pick a random topic and bounce it's leader + topic_index = randint(0, len(self.topics.keys()) - 1) + topic = self.topics.keys()[topic_index] + failures[failure_mode](self, topic, broker_type) + + def fail_many_brokers(self, failure_mode, num_failures): + sig = signal.SIGTERM + if (failure_mode == "clean_shutdown"): + sig = signal.SIGTERM + else: + sig = signal.SIGKILL + + for num in range(0, num_failures - 1): + signal_node(self, self.kafka.nodes[num], sig) + + + def setup_system(self): + # Setup phase + self.zk = ZookeeperService(self.test_context, num_nodes=1) + self.zk.start() + + self.kafka = KafkaService(self.test_context, num_nodes=self.replication, + zk=self.zk, topics=self.topics) + self.kafka.start() + # Start test harness + self.driver = StreamsSmokeTestDriverService(self.test_context, self.kafka) + self.processor1 = StreamsSmokeTestJobRunnerService(self.test_context, self.kafka) + + + self.driver.start() + self.processor1.start() + + def collect_results(self, sleep_time_secs): + data = {} + # End test + self.driver.wait() + self.driver.stop() + + self.processor1.stop() + + node = self.driver.node + + # Success is declared if streams does not crash when sleep time > 0 + # It should give an exception when sleep time is 0 since we kill the brokers immediately + # and the topic manager cannot create internal topics with the desired replication factor + if (sleep_time_secs == 0): + output_streams = self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-EXCEPTION %s" % self.processor1.STDOUT_FILE, allow_fail=False) + else: + output_streams = self.processor1.node.account.ssh_capture("grep SMOKE-TEST-CLIENT-CLOSED %s" % self.processor1.STDOUT_FILE, allow_fail=False) + + for line in output_streams: + data["Client closed"] = line + + # Currently it is hard to guarantee anything about Kafka since we don't have exactly once. + # With exactly once in place, success will be defined as ALL-RECORDS-DELIEVERD and SUCCESS + output = node.account.ssh_capture("grep -E 'ALL-RECORDS-DELIVERED|PROCESSED-MORE-THAN-GENERATED|PROCESSED-LESS-THAN-GENERATED' %s" % self.driver.STDOUT_FILE, allow_fail=False) + for line in output: + data["Records Delivered"] = line + output = node.account.ssh_capture("grep -E 'SUCCESS|FAILURE' %s" % self.driver.STDOUT_FILE, allow_fail=False) + for line in output: + data["Logic Success/Failure"] = line + + + return data + + @cluster(num_nodes=7) + @matrix(failure_mode=["clean_shutdown", "hard_shutdown"], + broker_type=["leader", "controller"], + sleep_time_secs=[120]) + def test_broker_type_bounce(self, failure_mode, broker_type, sleep_time_secs): + """ + Start a smoke test client, then kill one particular broker and ensure data is still received + Record if records are delivered. + """ + self.setup_system() + + # Sleep to allow test to run for a bit + time.sleep(sleep_time_secs) + + # Fail brokers + self.fail_broker_type(failure_mode, broker_type); + + return self.collect_results(sleep_time_secs) + + @cluster(num_nodes=7) + @matrix(failure_mode=["clean_shutdown"], + broker_type=["controller"], + sleep_time_secs=[0]) + def test_broker_type_bounce_at_start(self, failure_mode, broker_type, sleep_time_secs): + """ + Start a smoke test client, then kill one particular broker immediately before streams stats + Streams should throw an exception since it cannot create topics with the desired + replication factor of 3 + """ + self.setup_system() + + # Sleep to allow test to run for a bit + time.sleep(sleep_time_secs) + + # Fail brokers + self.fail_broker_type(failure_mode, broker_type); + + return self.collect_results(sleep_time_secs) + + @cluster(num_nodes=7) + @matrix(failure_mode=["clean_shutdown", "hard_shutdown"], + num_failures=[2]) + def test_many_brokers_bounce(self, failure_mode, num_failures): + """ + Start a smoke test client, then kill a few brokers and ensure data is still received + Record if records are delivered + """ + self.setup_system() + + # Sleep to allow test to run for a bit + time.sleep(120) + + # Fail brokers + self.fail_many_brokers(failure_mode, num_failures); + + return self.collect_results(120) diff --git a/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py new file mode 100644 index 0000000000000..d6e6c9a0d236e --- /dev/null +++ b/tests/kafkatest/tests/streams/streams_broker_compatibility_test.py @@ -0,0 +1,90 @@ +# 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. + +from ducktape.mark import parametrize +from ducktape.tests.test import Test + +from kafkatest.services.kafka import KafkaService +from kafkatest.services.streams import StreamsBrokerCompatibilityService +from kafkatest.services.verifiable_consumer import VerifiableConsumer +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.version import DEV_BRANCH, LATEST_0_10_1, LATEST_0_10_0, KafkaVersion + + +class StreamsBrokerCompatibility(Test): + """ + These tests validate that Streams v0.10.2+ can connect to older brokers v0.10.1+ + and that Streams fails fast for pre-0.10.0 brokers + """ + + input = "brokerCompatibilitySourceTopic" + output = "brokerCompatibilitySinkTopic" + + def __init__(self, test_context): + super(StreamsBrokerCompatibility, self).__init__(test_context=test_context) + + self.zk = ZookeeperService(test_context, num_nodes=1) + self.kafka = KafkaService(test_context, + num_nodes=1, + zk=self.zk, + topics={ + self.input: {'partitions': 1, 'replication-factor': 1}, + self.output: {'partitions': 1, 'replication-factor': 1} + }) + + self.processor = StreamsBrokerCompatibilityService(self.test_context, self.kafka) + + self.consumer = VerifiableConsumer(test_context, + 1, + self.kafka, + self.output, + "stream-broker-compatibility-verify-consumer") + + def setUp(self): + self.zk.start() + + @parametrize(broker_version=str(DEV_BRANCH)) + @parametrize(broker_version=str(LATEST_0_10_1)) + def test_compatible_brokers(self, broker_version): + self.kafka.set_version(KafkaVersion(broker_version)) + self.kafka.start() + + self.processor.start() + self.consumer.start() + + self.processor.wait() + + num_consumed_mgs = self.consumer.total_consumed() + + self.consumer.stop() + self.kafka.stop() + + assert num_consumed_mgs == 1, \ + "Did expect to read exactly one message but got %d" % num_consumed_mgs + + @parametrize(broker_version=str(LATEST_0_10_0)) + def test_fail_fast_on_incompatible_brokers(self, broker_version): + self.kafka.set_version(KafkaVersion(broker_version)) + self.kafka.start() + + self.processor.start() + + self.processor.node.account.ssh(self.processor.start_cmd(self.processor.node)) + with self.processor.node.account.monitor_log(self.processor.STDERR_FILE) as monitor: + monitor.wait_until('Exception in thread "main" org.apache.kafka.streams.errors.StreamsException: Kafka Streams requires broker version 0.10.1.x or higher.', + timeout_sec=60, + err_msg="Never saw 'incompatible broker' error message " + str(self.processor.node.account)) + + self.kafka.stop() diff --git a/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py b/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py index 5e4e7f28c683b..482da9c5d85f7 100644 --- a/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py +++ b/tests/kafkatest/tests/streams/streams_shutdown_deadlock_test.py @@ -13,11 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. -from ducktape.mark import ignore - from kafkatest.tests.kafka_test import KafkaTest from kafkatest.services.streams import StreamsSmokeTestShutdownDeadlockService + class StreamsShutdownDeadlockTest(KafkaTest): """ Simple test of Kafka Streams. diff --git a/tests/kafkatest/tests/streams/streams_smoke_test.py b/tests/kafkatest/tests/streams/streams_smoke_test.py index a824d92657323..496c495827551 100644 --- a/tests/kafkatest/tests/streams/streams_smoke_test.py +++ b/tests/kafkatest/tests/streams/streams_smoke_test.py @@ -27,7 +27,7 @@ class StreamsSmokeTest(KafkaTest): """ def __init__(self, test_context): - super(StreamsSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=2, topics={ + super(StreamsSmokeTest, self).__init__(test_context, num_zk=1, num_brokers=3, topics={ 'echo' : { 'partitions': 5, 'replication-factor': 1 }, 'data' : { 'partitions': 5, 'replication-factor': 1 }, 'min' : { 'partitions': 5, 'replication-factor': 1 }, @@ -46,7 +46,7 @@ def __init__(self, test_context): self.processor3 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) self.processor4 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) - @cluster(num_nodes=8) + @cluster(num_nodes=9) def test_streams(self): """ Start a few smoke test clients, then repeat start a new one, stop (cleanly) running one a few times. diff --git a/tests/kafkatest/tests/verifiable_consumer_test.py b/tests/kafkatest/tests/verifiable_consumer_test.py index 6417e615f5468..2ba2a619994f0 100644 --- a/tests/kafkatest/tests/verifiable_consumer_test.py +++ b/tests/kafkatest/tests/verifiable_consumer_test.py @@ -23,14 +23,16 @@ from kafkatest.services.kafka import TopicPartition class VerifiableConsumerTest(KafkaTest): + PRODUCER_REQUEST_TIMEOUT_SEC = 30 def __init__(self, test_context, num_consumers=1, num_producers=0, - group_id="test_group_id", session_timeout_sec=10, **kwargs): + group_id="test_group_id", session_timeout_sec=10, **kwargs): super(VerifiableConsumerTest, self).__init__(test_context, **kwargs) self.num_consumers = num_consumers self.num_producers = num_producers self.group_id = group_id self.session_timeout_sec = session_timeout_sec + self.consumption_timeout_sec = max(self.PRODUCER_REQUEST_TIMEOUT_SEC + 5, 2 * session_timeout_sec) def _all_partitions(self, topic, num_partitions): partitions = set() @@ -56,11 +58,14 @@ def min_cluster_size(self): def setup_consumer(self, topic, enable_autocommit=False, assignment_strategy="org.apache.kafka.clients.consumer.RangeAssignor"): return VerifiableConsumer(self.test_context, self.num_consumers, self.kafka, topic, self.group_id, session_timeout_sec=self.session_timeout_sec, - assignment_strategy=assignment_strategy, enable_autocommit=enable_autocommit) + assignment_strategy=assignment_strategy, enable_autocommit=enable_autocommit, + log_level="TRACE") def setup_producer(self, topic, max_messages=-1): return VerifiableProducer(self.test_context, self.num_producers, self.kafka, topic, - max_messages=max_messages, throughput=500) + max_messages=max_messages, throughput=500, + request_timeout_sec=self.PRODUCER_REQUEST_TIMEOUT_SEC, + log_level="DEBUG") def await_produced_messages(self, producer, min_messages=1000, timeout_sec=10): current_acked = producer.num_acked @@ -70,7 +75,7 @@ def await_produced_messages(self, producer, min_messages=1000, timeout_sec=10): def await_consumed_messages(self, consumer, min_messages=1): current_total = consumer.total_consumed() wait_until(lambda: consumer.total_consumed() >= current_total + min_messages, - timeout_sec=self.session_timeout_sec*2, + timeout_sec=self.consumption_timeout_sec, err_msg="Timed out waiting for consumption") def await_members(self, consumer, num_consumers): diff --git a/tests/kafkatest/version.py b/tests/kafkatest/version.py index 5dd09a1fa5150..7cd489d87ac64 100644 --- a/tests/kafkatest/version.py +++ b/tests/kafkatest/version.py @@ -31,11 +31,8 @@ class KafkaVersion(LooseVersion): assert v10 > v9 # assertion passes! """ def __init__(self, version_string): - self.is_trunk = (version_string.lower() == "trunk") - if self.is_trunk: - # Since "trunk" may actually be a branch that is not trunk, - # use kafkatest_version() for comparison purposes, - # and track whether we're in "trunk" with a flag + self.is_dev = (version_string.lower() == "dev") + if self.is_dev: version_string = kafkatest_version() # Drop dev suffix if present @@ -48,22 +45,22 @@ def __init__(self, version_string): LooseVersion.__init__(self, version_string) def __str__(self): - if self.is_trunk: - return "trunk" + if self.is_dev: + return "dev" else: return LooseVersion.__str__(self) def get_version(node=None): """Return the version attached to the given node. - Default to trunk if node or node.version is undefined (aka None) + Default to DEV_BRANCH if node or node.version is undefined (aka None) """ if node is not None and hasattr(node, "version") and node.version is not None: return node.version else: - return TRUNK + return DEV_BRANCH -TRUNK = KafkaVersion("trunk") +DEV_BRANCH = KafkaVersion("dev") # 0.8.2.X versions V_0_8_2_1 = KafkaVersion("0.8.2.1") @@ -82,6 +79,7 @@ def get_version(node=None): # 0.10.1.x versions V_0_10_1_0 = KafkaVersion("0.10.1.0") -LATEST_0_10_1 = V_0_10_1_0 +V_0_10_1_1 = KafkaVersion("0.10.1.1") +LATEST_0_10_1 = V_0_10_1_1 LATEST_0_10 = LATEST_0_10_1 diff --git a/tests/unit/directory_layout/check_project_paths.py b/tests/unit/directory_layout/check_project_paths.py index 284e4d5150c41..b9b76f13276be 100644 --- a/tests/unit/directory_layout/check_project_paths.py +++ b/tests/unit/directory_layout/check_project_paths.py @@ -16,7 +16,7 @@ from kafkatest.directory_layout.kafka_path import create_path_resolver, KafkaSystemTestPathResolver, \ KAFKA_PATH_RESOLVER_KEY -from kafkatest.version import V_0_9_0_1, TRUNK, KafkaVersion +from kafkatest.version import V_0_9_0_1, DEV_BRANCH, KafkaVersion class DummyContext(object): @@ -56,9 +56,9 @@ def check_paths(self): """Check expected path resolution without any version specified.""" resolver = create_path_resolver(DummyContext()) - assert resolver.home() == "/opt/kafka-trunk" - assert resolver.bin() == "/opt/kafka-trunk/bin" - assert resolver.script("kafka-run-class.sh") == "/opt/kafka-trunk/bin/kafka-run-class.sh" + assert resolver.home() == "/opt/kafka-dev" + assert resolver.bin() == "/opt/kafka-dev/bin" + assert resolver.script("kafka-run-class.sh") == "/opt/kafka-dev/bin/kafka-run-class.sh" def check_versioned_source_paths(self): """Check expected paths when using versions.""" @@ -74,16 +74,16 @@ def check_node_or_version_helper(self): """ resolver = create_path_resolver(DummyContext()) - # Node with no version attribute should resolve to TRUNK + # Node with no version attribute should resolve to DEV_BRANCH node = DummyNode() - assert resolver._version(node) == TRUNK + assert resolver._version(node) == DEV_BRANCH # Node with version attribute should resolve to the version attribute node.version = V_0_9_0_1 assert resolver._version(node) == V_0_9_0_1 # A KafkaVersion object should resolve to itself - assert resolver._version(TRUNK) == TRUNK + assert resolver._version(DEV_BRANCH) == DEV_BRANCH version = KafkaVersion("999.999.999") assert resolver._version(version) == version diff --git a/tests/unit/version/check_version.py b/tests/unit/version/check_version.py index 7b855518207da..8cf8e9a06e7b9 100644 --- a/tests/unit/version/check_version.py +++ b/tests/unit/version/check_version.py @@ -15,18 +15,18 @@ from mock import Mock -from kafkatest.version import TRUNK, V_0_8_2_2, get_version +from kafkatest.version import DEV_BRANCH, V_0_8_2_2, get_version class CheckVersion(object): def check_get_version(self): """Check default and override behavior of get_version""" node = None - assert get_version(node) == TRUNK + assert get_version(node) == DEV_BRANCH node = Mock() node.version = None - assert get_version(node) == TRUNK + assert get_version(node) == DEV_BRANCH node = Mock() node.version = V_0_8_2_2 diff --git a/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java new file mode 100644 index 0000000000000..6ed97bba0dee4 --- /dev/null +++ b/tools/src/main/java/org/apache/kafka/tools/ClientCompatibilityTest.java @@ -0,0 +1,377 @@ +/** + * 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.tools; + +import static net.sourceforge.argparse4j.impl.Arguments.store; + +import net.sourceforge.argparse4j.ArgumentParsers; +import net.sourceforge.argparse4j.inf.ArgumentParser; +import net.sourceforge.argparse4j.inf.ArgumentParserException; +import net.sourceforge.argparse4j.inf.Namespace; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.consumer.OffsetAndTimestamp; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.ClusterResource; +import org.apache.kafka.common.ClusterResourceListener; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.RecordTooLargeException; +import org.apache.kafka.common.errors.UnsupportedVersionException; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.HashMap; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Properties; +import java.util.concurrent.Future; + +/** + * ClientCompatibilityTest is invoked by the ducktape test client_compatibility_features_test.py to validate + * client behavior when various broker versions are in use. It runs various client operations and tests whether they + * are supported or not. + */ +public class ClientCompatibilityTest { + private static final Logger log = LoggerFactory.getLogger(ClientCompatibilityTest.class); + + static class TestConfig { + final String bootstrapServer; + final String topic; + final boolean offsetsForTimesSupported; + final boolean expectClusterId; + final boolean expectRecordTooLargeException; + + TestConfig(Namespace res) { + this.bootstrapServer = res.getString("bootstrapServer"); + this.topic = res.getString("topic"); + this.offsetsForTimesSupported = res.getBoolean("offsetsForTimesSupported"); + this.expectClusterId = res.getBoolean("clusterIdSupported"); + this.expectRecordTooLargeException = res.getBoolean("expectRecordTooLargeException"); + } + } + + public static void main(String[] args) throws Exception { + ArgumentParser parser = ArgumentParsers + .newArgumentParser("client-compatibility-test") + .defaultHelp(true) + .description("This tool is used to verify client compatibility guarantees."); + parser.addArgument("--topic") + .action(store()) + .required(true) + .type(String.class) + .dest("topic") + .metavar("TOPIC") + .help("the compatibility test will produce messages to this topic"); + parser.addArgument("--bootstrap-server") + .action(store()) + .required(true) + .type(String.class) + .dest("bootstrapServer") + .metavar("BOOTSTRAP_SERVER") + .help("The server(s) to use for bootstrapping"); + parser.addArgument("--offsets-for-times-supported") + .action(store()) + .required(true) + .type(Boolean.class) + .dest("offsetsForTimesSupported") + .metavar("OFFSETS_FOR_TIMES_SUPPORTED") + .help("True if KafkaConsumer#offsetsForTimes is supported by the current broker version"); + parser.addArgument("--cluster-id-supported") + .action(store()) + .required(true) + .type(Boolean.class) + .dest("clusterIdSupported") + .metavar("CLUSTER_ID_SUPPORTED") + .help("True if cluster IDs are supported. False if cluster ID always appears as null."); + parser.addArgument("--expect-record-too-large-exception") + .action(store()) + .required(true) + .type(Boolean.class) + .dest("expectRecordTooLargeException") + .metavar("EXPECT_RECORD_TOO_LARGE_EXCEPTION") + .help("True if we should expect a RecordTooLargeException when trying to read from a topic " + + "that contains a message that is bigger than " + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG + + ". This is pre-KIP-74 behavior."); + Namespace res = null; + try { + res = parser.parseArgs(args); + } catch (ArgumentParserException e) { + if (args.length == 0) { + parser.printHelp(); + System.exit(0); + } else { + parser.handleError(e); + System.exit(1); + } + } + TestConfig testConfig = new TestConfig(res); + ClientCompatibilityTest test = new ClientCompatibilityTest(testConfig); + try { + test.run(); + } catch (Throwable t) { + System.out.printf("FAILED: Caught exception %s\n\n", t.getMessage()); + t.printStackTrace(); + System.exit(1); + } + System.out.println("SUCCESS."); + System.exit(0); + } + + private static String toHexString(byte[] buf) { + StringBuilder bld = new StringBuilder(); + for (byte b : buf) { + bld.append(String.format("%02x", b)); + } + return bld.toString(); + } + + private static void compareArrays(byte[] a, byte[] b) { + if (!Arrays.equals(a, b)) { + throw new RuntimeException("Arrays did not match: expected " + toHexString(a) + ", got " + toHexString(b)); + } + } + + private final TestConfig testConfig; + + private final byte[] message1; + + private final byte[] message2; + + ClientCompatibilityTest(TestConfig testConfig) { + this.testConfig = testConfig; + long curTime = Time.SYSTEM.milliseconds(); + + ByteBuffer buf = ByteBuffer.allocate(8); + buf.putLong(curTime); + this.message1 = buf.array(); + + ByteBuffer buf2 = ByteBuffer.allocate(4096); + for (long i = 0; i < buf2.capacity(); i += 8) { + buf2.putLong(curTime + i); + } + this.message2 = buf2.array(); + } + + void run() throws Exception { + long prodTimeMs = Time.SYSTEM.milliseconds(); + testProduce(); + testConsume(prodTimeMs); + } + + public void testProduce() throws Exception { + Properties producerProps = new Properties(); + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, testConfig.bootstrapServer); + ByteArraySerializer serializer = new ByteArraySerializer(); + KafkaProducer producer = new KafkaProducer<>(producerProps, serializer, serializer); + ProducerRecord record1 = new ProducerRecord<>(testConfig.topic, message1); + Future future1 = producer.send(record1); + ProducerRecord record2 = new ProducerRecord<>(testConfig.topic, message2); + Future future2 = producer.send(record2); + producer.flush(); + future1.get(); + future2.get(); + producer.close(); + } + + private static class OffsetsForTime { + Map result; + + @Override + public String toString() { + return Utils.mkString(result); + } + } + + public static class ClientCompatibilityTestDeserializer implements Deserializer, ClusterResourceListener { + private final boolean expectClusterId; + + ClientCompatibilityTestDeserializer(boolean expectClusterId) { + this.expectClusterId = expectClusterId; + } + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public byte[] deserialize(String topic, byte[] data) { + return data; + } + + @Override + public void close() { + // nothing to do + } + + @Override + public void onUpdate(ClusterResource clusterResource) { + if (expectClusterId) { + if (clusterResource.clusterId() == null) { + throw new RuntimeException("Expected cluster id to be supported, but it was null."); + } + } else { + if (clusterResource.clusterId() != null) { + throw new RuntimeException("Expected cluster id to be null, but it was supported."); + } + } + } + } + + public void testConsume(final long prodTimeMs) throws Exception { + Properties consumerProps = new Properties(); + consumerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, testConfig.bootstrapServer); + consumerProps.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 512); + ClientCompatibilityTestDeserializer deserializer = + new ClientCompatibilityTestDeserializer(testConfig.expectClusterId); + final KafkaConsumer consumer = new KafkaConsumer<>(consumerProps, deserializer, deserializer); + final List partitionInfos = consumer.partitionsFor(testConfig.topic); + if (partitionInfos.size() < 1) + throw new RuntimeException("Expected at least one partition for topic " + testConfig.topic); + final Map timestampsToSearch = new HashMap<>(); + final LinkedList topicPartitions = new LinkedList<>(); + for (PartitionInfo partitionInfo : partitionInfos) { + TopicPartition topicPartition = new TopicPartition(partitionInfo.topic(), partitionInfo.partition()); + timestampsToSearch.put(topicPartition, prodTimeMs); + topicPartitions.add(topicPartition); + } + final OffsetsForTime offsetsForTime = new OffsetsForTime(); + tryFeature("offsetsForTimes", testConfig.offsetsForTimesSupported, + new Runnable() { + @Override + public void run() { + offsetsForTime.result = consumer.offsetsForTimes(timestampsToSearch); + } + }, + new Runnable() { + @Override + public void run() { + log.info("offsetsForTime = {}", offsetsForTime.result); + } + }); + // Whether or not offsetsForTimes works, beginningOffsets and endOffsets + // should work. + consumer.beginningOffsets(timestampsToSearch.keySet()); + consumer.endOffsets(timestampsToSearch.keySet()); + + consumer.assign(topicPartitions); + consumer.seekToBeginning(topicPartitions); + final Iterator iter = new Iterator() { + private final int timeoutMs = 10000; + private Iterator> recordIter = null; + private byte[] next = null; + + private byte[] fetchNext() { + while (true) { + long curTime = Time.SYSTEM.milliseconds(); + if (curTime - prodTimeMs > timeoutMs) + throw new RuntimeException("Timed out after " + timeoutMs + " ms."); + if (recordIter == null) { + ConsumerRecords records = consumer.poll(100); + recordIter = records.iterator(); + } + if (recordIter.hasNext()) + return recordIter.next().value(); + recordIter = null; + } + } + + @Override + public boolean hasNext() { + if (next != null) + return true; + next = fetchNext(); + return next != null; + } + + @Override + public byte[] next() { + if (!hasNext()) + throw new NoSuchElementException(); + byte[] cur = next; + next = null; + return cur; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + byte[] next = iter.next(); + try { + compareArrays(message1, next); + log.debug("Found first message..."); + } catch (RuntimeException e) { + throw new RuntimeException("The first message in this topic was not ours. Please use a new topic when " + + "running this program."); + } + try { + next = iter.next(); + if (testConfig.expectRecordTooLargeException) + throw new RuntimeException("Expected to get a RecordTooLargeException when reading a record " + + "bigger than " + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG); + try { + compareArrays(message2, next); + } catch (RuntimeException e) { + System.out.println("The second message in this topic was not ours. Please use a new " + + "topic when running this program."); + System.exit(1); + } + } catch (RecordTooLargeException e) { + log.debug("Got RecordTooLargeException", e); + if (!testConfig.expectRecordTooLargeException) + throw new RuntimeException("Got an unexpected RecordTooLargeException when reading a record " + + "bigger than " + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG); + } + log.debug("Closing consumer."); + consumer.close(); + log.info("Closed consumer."); + } + + private void tryFeature(String featureName, boolean supported, Runnable invoker, Runnable resultTester) { + try { + invoker.run(); + log.info("Successfully used feature {}", featureName); + } catch (UnsupportedVersionException e) { + log.info("Got UnsupportedVersionException when attempting to use feature {}", featureName); + if (supported) { + throw new RuntimeException("Expected " + featureName + " to be supported, but it wasn't.", e); + } + return; + } + if (!supported) { + throw new RuntimeException("Did not expect " + featureName + " to be supported, but it was."); + } + resultTester.run(); + } +} diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index a13d3ec68fc68..c277b83330767 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -3,9 +3,9 @@ * 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. @@ -14,11 +14,18 @@ import static net.sourceforge.argparse4j.impl.Arguments.store; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; + +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Properties; import java.util.Random; +import net.sourceforge.argparse4j.inf.MutuallyExclusiveGroup; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -42,15 +49,36 @@ public static void main(String[] args) throws Exception { /* parse args */ String topicName = res.getString("topic"); long numRecords = res.getLong("numRecords"); - int recordSize = res.getInt("recordSize"); + Integer recordSize = res.getInt("recordSize"); int throughput = res.getInt("throughput"); List producerProps = res.getList("producerConfig"); String producerConfig = res.getString("producerConfigFile"); + String payloadFilePath = res.getString("payloadFile"); + + // since default value gets printed with the help text, we are escaping \n there and replacing it with correct value here. + String payloadDelimiter = res.getString("payloadDelimiter").equals("\\n") ? "\n" : res.getString("payloadDelimiter"); if (producerProps == null && producerConfig == null) { throw new ArgumentParserException("Either --producer-props or --producer.config must be specified.", parser); } + List payloadByteList = new ArrayList<>(); + if (payloadFilePath != null) { + Path path = Paths.get(payloadFilePath); + System.out.println("Reading payloads from: " + path.toAbsolutePath()); + if (Files.notExists(path) || Files.size(path) == 0) { + throw new IllegalArgumentException("File does not exist or empty file provided."); + } + + String[] payloadList = new String(Files.readAllBytes(path), "UTF-8").split(payloadDelimiter); + + System.out.println("Number of messages read: " + payloadList.length); + + for (String payload : payloadList) { + payloadByteList.add(payload.getBytes(StandardCharsets.UTF_8)); + } + } + Properties props = new Properties(); if (producerConfig != null) { props.putAll(Utils.loadProps(producerConfig)); @@ -68,16 +96,24 @@ public static void main(String[] args) throws Exception { KafkaProducer producer = new KafkaProducer(props); /* setup perf test */ - byte[] payload = new byte[recordSize]; + byte[] payload = null; Random random = new Random(0); - for (int i = 0; i < payload.length; ++i) - payload[i] = (byte) (random.nextInt(26) + 65); - ProducerRecord record = new ProducerRecord<>(topicName, payload); + if (recordSize != null) { + payload = new byte[recordSize]; + for (int i = 0; i < payload.length; ++i) + payload[i] = (byte) (random.nextInt(26) + 65); + } + ProducerRecord record; Stats stats = new Stats(numRecords, 5000); long startMs = System.currentTimeMillis(); ThroughputThrottler throttler = new ThroughputThrottler(throughput, startMs); for (int i = 0; i < numRecords; i++) { + if (payloadFilePath != null) { + payload = payloadByteList.get(random.nextInt(payloadByteList.size())); + } + record = new ProducerRecord<>(topicName, payload); + long sendStartMs = System.currentTimeMillis(); Callback cb = stats.nextCompletion(sendStartMs, payload.length, stats); producer.send(record, cb); @@ -109,6 +145,11 @@ private static ArgumentParser argParser() { .defaultHelp(true) .description("This tool is used to verify the producer performance."); + MutuallyExclusiveGroup payloadOptions = parser + .addMutuallyExclusiveGroup() + .required(true) + .description("either --record-size or --payload-file must be specified but not both."); + parser.addArgument("--topic") .action(store()) .required(true) @@ -124,13 +165,34 @@ private static ArgumentParser argParser() { .dest("numRecords") .help("number of messages to produce"); - parser.addArgument("--record-size") + payloadOptions.addArgument("--record-size") .action(store()) - .required(true) + .required(false) .type(Integer.class) .metavar("RECORD-SIZE") .dest("recordSize") - .help("message size in bytes"); + .help("message size in bytes. Note that you must provide exactly one of --record-size or --payload-file."); + + payloadOptions.addArgument("--payload-file") + .action(store()) + .required(false) + .type(String.class) + .metavar("PAYLOAD-FILE") + .dest("payloadFile") + .help("file to read the message payloads from. This works only for UTF-8 encoded text files. " + + "Payloads will be read from this file and a payload will be randomly selected when sending messages. " + + "Note that you must provide exactly one of --record-size or --payload-file."); + + parser.addArgument("--payload-delimiter") + .action(store()) + .required(false) + .type(String.class) + .metavar("PAYLOAD-DELIMITER") + .dest("payloadDelimiter") + .setDefault("\\n") + .help("provides delimiter to be used when --payload-file is provided. " + + "Defaults to new line. " + + "Note that this parameter will be ignored if --payload-file is not provided."); parser.addArgument("--throughput") .action(store()) diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java index daf569cd77173..faa9698e5a48d 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableLog4jAppender.java @@ -158,7 +158,7 @@ private static ArgumentParser argParser() { * * Note: this duplication of org.apache.kafka.common.utils.Utils.loadProps is unfortunate * but *intentional*. In order to use VerifiableProducer in compatibility and upgrade tests, - * we use VerifiableProducer from trunk tools package, and run it against 0.8.X.X kafka jars. + * we use VerifiableProducer from the development tools package, and run it against 0.8.X.X kafka jars. * Since this method is not in Utils in the 0.8.X.X jars, we have to cheat a bit and duplicate. */ public static Properties loadProps(String filename) throws IOException, FileNotFoundException { diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 54d53f10bea13..e81eb8f3ae01b 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -160,7 +160,7 @@ private static ArgumentParser argParser() { * * Note: this duplication of org.apache.kafka.common.utils.Utils.loadProps is unfortunate * but *intentional*. In order to use VerifiableProducer in compatibility and upgrade tests, - * we use VerifiableProducer from trunk tools package, and run it against 0.8.X.X kafka jars. + * we use VerifiableProducer from the development tools package, and run it against 0.8.X.X kafka jars. * Since this method is not in Utils in the 0.8.X.X jars, we have to cheat a bit and duplicate. */ public static Properties loadProps(String filename) throws IOException, FileNotFoundException { diff --git a/vagrant/base.sh b/vagrant/base.sh index 369776584a2fd..448c640d2bfc9 100755 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -37,11 +37,11 @@ if [ -z `which javac` ]; then fi chmod a+rw /opt -if [ -h /opt/kafka-trunk ]; then +if [ -h /opt/kafka-dev ]; then # reset symlink - rm /opt/kafka-trunk + rm /opt/kafka-dev fi -ln -s /vagrant /opt/kafka-trunk +ln -s /vagrant /opt/kafka-dev get_kafka() { version=$1 @@ -67,6 +67,8 @@ get_kafka 0.9.0.1 chmod a+rw /opt/kafka-0.9.0.1 get_kafka 0.10.0.1 chmod a+rw /opt/kafka-0.10.0.1 +get_kafka 0.10.1.1 +chmod a+rw /opt/kafka-0.10.1.1 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local diff --git a/vagrant/broker.sh b/vagrant/broker.sh index 06dee6b97abec..986f0fae43650 100755 --- a/vagrant/broker.sh +++ b/vagrant/broker.sh @@ -23,7 +23,7 @@ PUBLIC_ADDRESS=$2 PUBLIC_ZOOKEEPER_ADDRESSES=$3 JMX_PORT=$4 -kafka_dir=/opt/kafka-trunk +kafka_dir=/opt/kafka-dev cd $kafka_dir sed \ diff --git a/vagrant/zk.sh b/vagrant/zk.sh index 67924d129d98e..e8c690a060e32 100755 --- a/vagrant/zk.sh +++ b/vagrant/zk.sh @@ -22,7 +22,7 @@ ZKID=$1 NUM_ZK=$2 JMX_PORT=$3 -kafka_dir=/opt/kafka-trunk +kafka_dir=/opt/kafka-dev cd $kafka_dir cp $kafka_dir/config/zookeeper.properties $kafka_dir/config/zookeeper-$ZKID.properties