From bf788846eef60cd98a971cd9c1e8f46466cd2123 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Wed, 20 Nov 2019 17:20:04 -0800 Subject: [PATCH 01/21] [KAFKA-6144]: Allow serving interactive queries from in-sync Standbys - Rough initial cut, with basic HA working. Many more changes needed still - [TODO] Introduce a new version for AssignmentInfo & support version probing - [TODO] Implement API changes in KafkaStreams (deprecation, new classes, new methods) - [TODO] Tests using metadataForKey(), redone over new APIs. New tests with standbys involved. - [TODO] Tests need to be (re)written, including version probing ones --- .../apache/kafka/streams/KafkaStreams.java | 23 ++- .../apache/kafka/streams/StreamsConfig.java | 4 +- .../processor/internals/StreamThread.java | 4 + .../internals/StreamsMetadataState.java | 131 ++++++++++++++---- .../internals/StreamsPartitionAssignor.java | 36 +++-- .../processor/internals/TaskManager.java | 5 +- .../internals/assignment/AssignmentInfo.java | 57 ++++++-- .../StreamThreadStateStoreProvider.java | 14 +- .../internals/StreamsMetadataStateTest.java | 24 ++-- .../StreamsPartitionAssignorTest.java | 37 +++-- .../assignment/AssignmentInfoTest.java | 30 ++-- .../StreamThreadStateStoreProviderTest.java | 2 +- .../streams/tests/StreamsUpgradeTest.java | 3 +- 13 files changed, 280 insertions(+), 90 deletions(-) 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 2a7470a3999e8..bbcb19058dc72 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1020,7 +1020,7 @@ public void cleanUp() { */ public Collection allMetadata() { validateIsRunning(); - return streamsMetadataState.getAllMetadata(); + return streamsMetadataState.getAllActiveMetadata(); } /** @@ -1073,7 +1073,7 @@ public Collection allMetadataForStore(final String storeName) { * @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 + * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@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, @@ -1104,7 +1104,7 @@ public StreamsMetadata metadataForKey(final String storeName, * @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 + * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@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, @@ -1114,6 +1114,23 @@ public StreamsMetadata metadataForKey(final String storeName, return streamsMetadataState.getMetadataWithKey(storeName, key, partitioner); } + /** + * [TENTATIVE] Provide metadata for all hosts that have this key. + * + * @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 List} for the key ordered active first and then standby or + * {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing + */ + public List allMetadataWithKey(final String storeName, + final K key, + final Serializer keySerializer) { + validateIsRunning(); + return streamsMetadataState.getAllMetadataWithKey(storeName, key, keySerializer); + } + /** * 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}. 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 cdec8179877e2..e445a2c97223d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -302,10 +302,10 @@ public class StreamsConfig extends AbstractConfig { public static final String APPLICATION_ID_CONFIG = "application.id"; 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."; - /**{@code user.endpoint} */ + /**{@code application.server} */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_SERVER_CONFIG = "application.server"; - 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"; + private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to an embedded user defined endpoint that can be used for state store discovery and interactive queries within a single KafkaStreams application"; /** {@code bootstrap.servers} */ @SuppressWarnings("WeakerAccess") 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 5abdef1f451ed..2902ef452f6a2 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 @@ -1195,6 +1195,10 @@ public Map tasks() { return taskManager.activeTasks(); } + public Map standbyTasks() { + return taskManager.standbyTasks(); + } + /** * Produces a string representation containing useful information about a StreamThread. * This is useful in debugging scenarios. 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 27826a63ff78f..e3d01e17f2f9e 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.Arrays; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -44,7 +45,8 @@ public class StreamsMetadataState { public static final HostInfo UNKNOWN_HOST = new HostInfo("unknown", -1); private final InternalTopologyBuilder builder; - private final List allMetadata = new ArrayList<>(); + private final List allActiveMetadata = new ArrayList<>(); + private final List allStandbyMetadata = new ArrayList<>(); private final Set globalStores; private final HostInfo thisHost; private Cluster clusterMetadata; @@ -63,7 +65,8 @@ public String toString() { public String toString(final String indent) { final StringBuilder builder = new StringBuilder(); - builder.append(indent).append("GlobalMetadata: ").append(allMetadata).append("\n"); + builder.append(indent).append("GlobalActiveMetadata: ").append(allActiveMetadata).append("\n"); + builder.append(indent).append("GlobalStandbyMetadata: ").append(allStandbyMetadata).append("\n"); builder.append(indent).append("GlobalStores: ").append(globalStores).append("\n"); builder.append(indent).append("My HostInfo: ").append(thisHost).append("\n"); builder.append(indent).append(clusterMetadata).append("\n"); @@ -77,8 +80,8 @@ public String toString(final String indent) { * * @return all the {@link StreamsMetadata}s in a {@link KafkaStreams} application */ - public synchronized Collection getAllMetadata() { - return allMetadata; + public synchronized Collection getAllActiveMetadata() { + return allActiveMetadata; } /** @@ -95,7 +98,7 @@ public synchronized Collection getAllMetadataForStore(final Str } if (globalStores.contains(storeName)) { - return allMetadata; + return allActiveMetadata; } final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); @@ -104,7 +107,7 @@ public synchronized Collection getAllMetadataForStore(final Str } final ArrayList results = new ArrayList<>(); - for (final StreamsMetadata metadata : allMetadata) { + for (final StreamsMetadata metadata : allActiveMetadata) { if (metadata.stateStoreNames().contains(storeName)) { results.add(metadata); } @@ -142,7 +145,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return allMetadata.get(0); + return allActiveMetadata.get(0); } return myMetadata; } @@ -158,6 +161,39 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam sourceTopicsInfo); } + /** + * TODO: rework based on final API design + */ + public synchronized List getAllMetadataWithKey(final String storeName, + final K key, + final Serializer keySerializer) { + Objects.requireNonNull(keySerializer, "keySerializer can't be null"); + Objects.requireNonNull(storeName, "storeName can't be null"); + Objects.requireNonNull(key, "key can't be null"); + + if (!isInitialized()) { + return Arrays.asList(StreamsMetadata.NOT_AVAILABLE); + } + + if (globalStores.contains(storeName)) { + // global stores are on every node. if we dont' have the host info + // for this host then just pick the first metadata + if (thisHost == UNKNOWN_HOST) { + return allActiveMetadata; + } + return Arrays.asList(myMetadata); + } + + final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); + if (sourceTopicsInfo == null) { + return null; + } + + return getAllStreamsMetadataForKey(storeName, + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), + sourceTopicsInfo); + } @@ -190,7 +226,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return allMetadata.get(0); + return allActiveMetadata.get(0); } return myMetadata; } @@ -205,12 +241,14 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam /** * Respond to changes to the HostInfo -> TopicPartition mapping. Will rebuild the * metadata - * @param currentState the current mapping of {@link HostInfo} -> {@link TopicPartition}s + * @param activePartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for active partitions + * @param standbyPartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for standby partitions * @param clusterMetadata the current clusterMetadata {@link Cluster} */ - synchronized void onChange(final Map> currentState, final Cluster clusterMetadata) { + synchronized void onChange(final Map> activePartitionHostMap, + final Map> standbyPartitionHostMap, final Cluster clusterMetadata) { this.clusterMetadata = clusterMetadata; - rebuildMetadata(currentState); + rebuildMetadata(activePartitionHostMap, standbyPartitionHostMap); } private boolean hasPartitionsForAnyTopics(final List topicNames, final Set partitionForHost) { @@ -222,29 +260,43 @@ private boolean hasPartitionsForAnyTopics(final List topicNames, final S return false; } - private void rebuildMetadata(final Map> currentState) { - allMetadata.clear(); - if (currentState.isEmpty()) { + private Set getStoresOnHost(final Map> stores, final Set partitionsForHost) { + final Set storesOnHost = new HashSet<>(); + for (final Map.Entry> storeTopicEntry : stores.entrySet()) { + final List topicsForStore = storeTopicEntry.getValue(); + if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) { + storesOnHost.add(storeTopicEntry.getKey()); + } + } + return storesOnHost; + } + + + private void rebuildMetadata(final Map> activePartitionHostMap, + final Map> standbyPartitionHostMap) { + allActiveMetadata.clear(); + allStandbyMetadata.clear(); + if (activePartitionHostMap.isEmpty()) { return; } final Map> stores = builder.stateStoreNameToSourceTopics(); - for (final Map.Entry> entry : currentState.entrySet()) { + for (final Map.Entry> entry : activePartitionHostMap.entrySet()) { final HostInfo key = entry.getKey(); final Set partitionsForHost = new HashSet<>(entry.getValue()); - final Set storesOnHost = new HashSet<>(); - for (final Map.Entry> storeTopicEntry : stores.entrySet()) { - final List topicsForStore = storeTopicEntry.getValue(); - if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) { - storesOnHost.add(storeTopicEntry.getKey()); - } - } + final Set storesOnHost = getStoresOnHost(stores, partitionsForHost); storesOnHost.addAll(globalStores); final StreamsMetadata metadata = new StreamsMetadata(key, storesOnHost, partitionsForHost); - allMetadata.add(metadata); + allActiveMetadata.add(metadata); if (key.equals(thisHost)) { myMetadata = metadata; } } + for (final Map.Entry> entry : standbyPartitionHostMap.entrySet()) { + final Set partitionsForHost = new HashSet<>(entry.getValue()); + final Set storesOnHost = getStoresOnHost(stores, partitionsForHost); + final StreamsMetadata metadata = new StreamsMetadata(entry.getKey(), storesOnHost, partitionsForHost); + allStandbyMetadata.add(metadata); + } } private StreamsMetadata getStreamsMetadataForKey(final String storeName, @@ -258,7 +310,7 @@ private StreamsMetadata getStreamsMetadataForKey(final String storeName, matchingPartitions.add(new TopicPartition(sourceTopic, partition)); } - for (final StreamsMetadata streamsMetadata : allMetadata) { + for (final StreamsMetadata streamsMetadata : allActiveMetadata) { final Set stateStoreNames = streamsMetadata.stateStoreNames(); final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); topicPartitions.retainAll(matchingPartitions); @@ -270,6 +322,37 @@ private StreamsMetadata getStreamsMetadataForKey(final String storeName, return null; } + private List getMatchingMetadata(final Set matchingPartitions, + final List allMetadata, final String storeName) { + final List matchingMetadata = new ArrayList<>(); + for (final StreamsMetadata streamsMetadata : allMetadata) { + final Set stateStoreNames = streamsMetadata.stateStoreNames(); + final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); + topicPartitions.retainAll(matchingPartitions); + if (stateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { + matchingMetadata.add(streamsMetadata); + } + } + return matchingMetadata; + } + + private List getAllStreamsMetadataForKey(final String storeName, + final K key, + final StreamPartitioner partitioner, + final SourceTopicsInfo sourceTopicsInfo) { + + final Integer partition = partitioner.partition(sourceTopicsInfo.topicWithMostPartitions, key, null, sourceTopicsInfo.maxPartitions); + final Set matchingPartitions = new HashSet<>(); + for (final String sourceTopic : sourceTopicsInfo.sourceTopics) { + matchingPartitions.add(new TopicPartition(sourceTopic, partition)); + } + + final List matchingMetadata = new ArrayList<>(); + matchingMetadata.addAll(getMatchingMetadata(matchingPartitions, allActiveMetadata, storeName)); + matchingMetadata.addAll(getMatchingMetadata(matchingPartitions, allStandbyMetadata, storeName)); + return matchingMetadata; + } + private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); if (sourceTopics == null || sourceTopics.isEmpty()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 6cb20536f1328..086396d9c454c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -283,6 +283,7 @@ private Map errorAssignment(final Map Collections.emptyList(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), errorCode).encode() )); } @@ -620,7 +621,8 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // ---------------- Step Three ---------------- // // construct the global partition assignment per host map - final Map> partitionsByHostState = new HashMap<>(); + final Map> partitionsByHost = new HashMap<>(); + final Map> standbyPartitionsByHost = new HashMap<>(); if (minReceivedMetadataVersion >= 2) { for (final Map.Entry entry : clientMetadataMap.entrySet()) { final HostInfo hostInfo = entry.getValue().hostInfo; @@ -628,24 +630,32 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // if application server is configured, also include host state map if (hostInfo != null) { final Set topicPartitions = new HashSet<>(); + final Set standbyPartitions = new HashSet<>(); final ClientState state = entry.getValue().state; for (final TaskId id : state.activeTasks()) { topicPartitions.addAll(partitionsForTask.get(id)); } - partitionsByHostState.put(hostInfo, topicPartitions); + // TODO: why do we have standby tasks even for non stateful sub topologies? + for (final TaskId id : state.standbyTasks()) { + standbyPartitions.addAll(partitionsForTask.get(id)); + } + + partitionsByHost.put(hostInfo, topicPartitions); + standbyPartitionsByHost.put(hostInfo, standbyPartitions); } } } - taskManager.setPartitionsByHostState(partitionsByHostState); + taskManager.setHostPartitionMappings(partitionsByHost, standbyPartitionsByHost); final Map assignment; if (versionProbing) { assignment = versionProbingAssignment( clientMetadataMap, partitionsForTask, - partitionsByHostState, + partitionsByHost, + standbyPartitionsByHost, allOwnedPartitions, minReceivedMetadataVersion, minSupportedMetadataVersion @@ -654,7 +664,8 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr assignment = computeNewAssignment( clientMetadataMap, partitionsForTask, - partitionsByHostState, + partitionsByHost, + standbyPartitionsByHost, allOwnedPartitions, minReceivedMetadataVersion, minSupportedMetadataVersion @@ -667,6 +678,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr private Map computeNewAssignment(final Map clientsMetadata, final Map> partitionsForTask, final Map> partitionsByHostState, + final Map> standbyPartitionsByHost, final Set allOwnedPartitions, final int minUserMetadataVersion, final int minSupportedMetadataVersion) { @@ -700,6 +712,7 @@ private Map computeNewAssignment(final Map computeNewAssignment(final Map versionProbingAssignment(final Map clientsMetadata, final Map> partitionsForTask, - final Map> partitionsByHostState, + final Map> partitionsByHost, + final Map> standbyPartitionsByHost, final Set allOwnedPartitions, final int minUserMetadataVersion, final int minSupportedMetadataVersion) { @@ -734,7 +748,8 @@ private Map versionProbingAssignment(final Map assignment, final ClientMetadata clientMetadata, final Map> partitionsForTask, final Map> partitionsByHostState, + final Map> standbyPartitionsByHost, final Set allOwnedPartitions, final Map> activeTaskAssignments, final Map> standbyTaskAssignments, @@ -785,6 +801,7 @@ private void addClientAssignments(final Map assignment, assignedActiveList, standbyTaskMap, partitionsByHostState, + standbyPartitionsByHost, AssignorError.NONE.code() ).encode() ) @@ -1113,6 +1130,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat // version 2 fields final Map topicToPartitionInfo = new HashMap<>(); final Map> partitionsByHost; + final Map> standbyPartitionsByHost; final Map partitionsToTaskId = new HashMap<>(); @@ -1120,6 +1138,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat case 1: processVersionOneAssignment(logPrefix, info, partitions, activeTasks, partitionsToTaskId); partitionsByHost = Collections.emptyMap(); + standbyPartitionsByHost = Collections.emptyMap(); break; case 2: case 3: @@ -1127,6 +1146,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat case 5: processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); + standbyPartitionsByHost = info.standbyPartitionByHost(); break; default: throw new IllegalStateException( @@ -1136,7 +1156,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat } taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo)); - taskManager.setPartitionsByHostState(partitionsByHost); + taskManager.setHostPartitionMappings(partitionsByHost, standbyPartitionsByHost); taskManager.setPartitionsToTaskId(partitionsToTaskId); taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks()); taskManager.updateSubscriptionsFromAssignment(partitions); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index 43284c00a286c..a2ddf9a6f181e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -440,8 +440,9 @@ public void setClusterMetadata(final Cluster cluster) { this.cluster = cluster; } - public void setPartitionsByHostState(final Map> partitionsByHostState) { - this.streamsMetadataState.onChange(partitionsByHostState, cluster); + public void setHostPartitionMappings(final Map> partitionsByHost, + final Map> standbyPartitionsByHost) { + this.streamsMetadataState.onChange(partitionsByHost, standbyPartitionsByHost, cluster); } public void setPartitionsToTaskId(final Map partitionsToTaskId) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index a9ce225ef477e..691afcb83eebb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -49,6 +49,7 @@ public class AssignmentInfo { private List activeTasks; private Map> standbyTasks; private Map> partitionsByHost; + private Map> standbyPartitionsByHost; // used for decoding and "future consumer" assignments during version probing public AssignmentInfo(final int version, @@ -58,6 +59,7 @@ public AssignmentInfo(final int version, Collections.emptyList(), Collections.emptyMap(), Collections.emptyMap(), + Collections.emptyMap(), 0); } @@ -65,8 +67,9 @@ public AssignmentInfo(final int version, final List activeTasks, final Map> standbyTasks, final Map> partitionsByHost, + final Map> standbyPartitionsByHost, final int errCode) { - this(version, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, partitionsByHost, errCode); + this(version, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, partitionsByHost, standbyPartitionsByHost, errCode); } public AssignmentInfo(final int version, @@ -74,12 +77,14 @@ public AssignmentInfo(final int version, final List activeTasks, final Map> standbyTasks, final Map> partitionsByHost, + final Map> standbyPartitionsByHost, final int errCode) { this.usedVersion = version; this.commonlySupportedVersion = commonlySupportedVersion; this.activeTasks = activeTasks; this.standbyTasks = standbyTasks; this.partitionsByHost = partitionsByHost; + this.standbyPartitionsByHost = standbyPartitionsByHost; this.errCode = errCode; if (version < 1 || version > LATEST_SUPPORTED_VERSION) { @@ -112,6 +117,10 @@ public Map> partitionsByHost() { return partitionsByHost; } + public Map> standbyPartitionByHost() { + return standbyPartitionsByHost; + } + /** * @throws TaskAssignmentException if method fails to encode the data, e.g., if there is an * IO exception during encoding @@ -148,6 +157,7 @@ public ByteBuffer encode() { out.writeInt(commonlySupportedVersion); encodeActiveAndStandbyTaskAssignment(out); encodePartitionsByHostAsDictionary(out); + encodeStandbyPartitionsByHostAsDictionary(out); out.writeInt(errCode); break; default: @@ -191,12 +201,13 @@ private void encodePartitionsByHost(final DataOutputStream out) throws IOExcepti } } - private void encodePartitionsByHostAsDictionary(final DataOutputStream out) throws IOException { + private void encodeHostPartitionMapAsDictionary(final DataOutputStream out, + final Map> hostPartitionMap) throws IOException { // Build a dictionary to encode topicNames int topicIndex = 0; final Map topicNameDict = new HashMap<>(); - for (final Map.Entry> entry : partitionsByHost.entrySet()) { + for (final Map.Entry> entry : hostPartitionMap.entrySet()) { for (final TopicPartition topicPartition : entry.getValue()) { if (!topicNameDict.containsKey(topicPartition.topic())) { topicNameDict.put(topicPartition.topic(), topicIndex++); @@ -212,10 +223,10 @@ private void encodePartitionsByHostAsDictionary(final DataOutputStream out) thro } // encode partitions by host - out.writeInt(partitionsByHost.size()); + out.writeInt(hostPartitionMap.size()); // Write the topic index, partition - for (final Map.Entry> entry : partitionsByHost.entrySet()) { + for (final Map.Entry> entry : hostPartitionMap.entrySet()) { writeHostInfo(out, entry.getKey()); out.writeInt(entry.getValue().size()); for (final TopicPartition partition : entry.getValue()) { @@ -225,6 +236,14 @@ private void encodePartitionsByHostAsDictionary(final DataOutputStream out) thro } } + private void encodePartitionsByHostAsDictionary(final DataOutputStream out) throws IOException { + encodeHostPartitionMapAsDictionary(out, partitionsByHost); + } + + private void encodeStandbyPartitionsByHostAsDictionary(final DataOutputStream out) throws IOException { + encodeHostPartitionMapAsDictionary(out, standbyPartitionsByHost); + } + private void writeHostInfo(final DataOutputStream out, final HostInfo hostInfo) throws IOException { out.writeUTF(hostInfo.host()); out.writeInt(hostInfo.port()); @@ -285,6 +304,7 @@ public static AssignmentInfo decode(final ByteBuffer data) { decodeActiveTasks(assignmentInfo, in); decodeStandbyTasks(assignmentInfo, in); decodePartitionsByHostUsingDictionary(assignmentInfo, in); + decodeStandbyPartitionsByHostUsingDictionary(assignmentInfo, in); assignmentInfo.errCode = in.readInt(); break; default: @@ -338,9 +358,9 @@ private static Set readTopicPartitions(final DataInputStream in) return partitions; } - private static void decodePartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, + private static Map> decodeHostPartitionMapUsingDictionary(final AssignmentInfo assignmentInfo, final DataInputStream in) throws IOException { - assignmentInfo.partitionsByHost = new HashMap<>(); + final Map> hostPartitionMap = new HashMap<>(); final int dictSize = in.readInt(); final Map topicIndexDict = new HashMap<>(dictSize); for (int i = 0; i < dictSize; i++) { @@ -350,8 +370,19 @@ private static void decodePartitionsByHostUsingDictionary(final AssignmentInfo a final int numEntries = in.readInt(); for (int i = 0; i < numEntries; i++) { final HostInfo hostInfo = new HostInfo(in.readUTF(), in.readInt()); - assignmentInfo.partitionsByHost.put(hostInfo, readTopicPartitions(in, topicIndexDict)); + hostPartitionMap.put(hostInfo, readTopicPartitions(in, topicIndexDict)); } + return hostPartitionMap; + } + + private static void decodePartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, + final DataInputStream in) throws IOException { + assignmentInfo.partitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); + } + + private static void decodeStandbyPartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, + final DataInputStream in) throws IOException { + assignmentInfo.standbyPartitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); } private static Set readTopicPartitions(final DataInputStream in, @@ -366,8 +397,9 @@ private static Set readTopicPartitions(final DataInputStream in, @Override public int hashCode() { + final int hostMapHashCode = partitionsByHost.hashCode() ^ standbyPartitionsByHost.hashCode(); return usedVersion ^ commonlySupportedVersion ^ activeTasks.hashCode() ^ standbyTasks.hashCode() - ^ partitionsByHost.hashCode() ^ errCode; + ^ hostMapHashCode ^ errCode; } @Override @@ -379,7 +411,8 @@ public boolean equals(final Object o) { errCode == other.errCode && activeTasks.equals(other.activeTasks) && standbyTasks.equals(other.standbyTasks) && - partitionsByHost.equals(other.partitionsByHost); + partitionsByHost.equals(other.partitionsByHost) && + standbyPartitionsByHost.equals(other.standbyPartitionsByHost); } else { return false; } @@ -391,6 +424,8 @@ public String toString() { + ", supported version=" + commonlySupportedVersion + ", active tasks=" + activeTasks + ", standby tasks=" + standbyTasks - + ", partitions by host=" + partitionsByHost + "]"; + + ", partitions by host=" + partitionsByHost + + ", standbyPartitions by host=" + standbyPartitionsByHost + + "]"; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index 53c5cc0210672..2bdc5a9c51da6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.state.internals; +import java.util.HashSet; +import java.util.Set; import org.apache.kafka.streams.errors.InvalidStateStoreException; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.internals.StreamThread; @@ -46,12 +48,18 @@ public List stores(final String storeName, final QueryableStoreType qu if (streamThread.state() == StreamThread.State.DEAD) { return Collections.emptyList(); } - if (!streamThread.isRunningAndNotRebalancing()) { + // TODO: This needs to be rethought + if (!streamThread.isRunning()) { throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the stream thread is " + - streamThread.state() + ", not RUNNING"); + streamThread.state() + ", not RUNNING"); } final List stores = new ArrayList<>(); - for (final Task streamTask : streamThread.tasks().values()) { + final Set tasks = new HashSet<>(streamThread.tasks().values()); + if (streamThread.standbyTasks() != null) { + tasks.addAll(streamThread.standbyTasks().values()); + } + + for (final Task streamTask : tasks) { final StateStore store = streamTask.getStore(storeName); if (store != null && queryableStoreType.accepts(store)) { if (!store.isOpen()) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index bdc7a65c3866b..9002c03284665 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -122,7 +122,7 @@ public Object apply(final Object value) { cluster = new Cluster(null, Collections.emptyList(), partitionInfos, Collections.emptySet(), Collections.emptySet()); metadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne); - metadataState.onChange(hostToPartitions, cluster); + metadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); partitioner = new StreamPartitioner() { @Override public Integer partition(final String topic, final String key, final Object value, final int numPartitions) { @@ -145,7 +145,7 @@ public void shouldGetAllStreamInstances() { final StreamsMetadata three = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), Collections.singleton(topic3P0)); - final Collection actual = metadataState.getAllMetadata(); + final Collection actual = metadataState.getAllActiveMetadata(); assertEquals(3, actual.size()); assertTrue("expected " + actual + " to contain " + one, actual.contains(one)); assertTrue("expected " + actual + " to contain " + two, actual.contains(two)); @@ -165,11 +165,12 @@ public boolean test(final Object key, final Object value) { final HostInfo hostFour = new HostInfo("host-four", 8080); hostToPartitions.put(hostFour, Utils.mkSet(tp5)); - metadataState.onChange(hostToPartitions, cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); + metadataState.onChange(hostToPartitions, Collections.emptyMap(), + cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostFour, Collections.singleton(globalTable), Collections.singleton(tp5)); - final Collection actual = metadataState.getAllMetadata(); + final Collection actual = metadataState.getAllActiveMetadata(); assertTrue("expected " + actual + " to contain " + expected, actual.contains(expected)); } @@ -201,7 +202,8 @@ public void shouldGetInstanceWithKey() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); - metadataState.onChange(hostToPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); + metadataState.onChange(hostToPartitions, Collections.emptyMap(), + cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), Collections.singleton(topic3P0)); @@ -218,7 +220,8 @@ public void shouldGetInstanceWithKeyAndCustomPartitioner() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); - metadataState.onChange(hostToPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); + metadataState.onChange(hostToPartitions, Collections.emptyMap(), + cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-three", "merged-table"), Utils.mkSet(topic2P0, tp4)); @@ -229,7 +232,7 @@ public void shouldGetInstanceWithKeyAndCustomPartitioner() { @Test public void shouldReturnNotAvailableWhenClusterIsEmpty() { - metadataState.onChange(Collections.>emptyMap(), Cluster.empty()); + metadataState.onChange(Collections.>emptyMap(), Collections.emptyMap(), Cluster.empty()); final StreamsMetadata result = metadataState.getMetadataWithKey("table-one", "a", Serdes.String().serializer()); assertEquals(StreamsMetadata.NOT_AVAILABLE, result); } @@ -238,7 +241,8 @@ public void shouldReturnNotAvailableWhenClusterIsEmpty() { public void shouldGetInstanceWithKeyWithMergedStreams() { final TopicPartition topic2P2 = new TopicPartition("topic-two", 2); hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1, topic2P2)); - metadataState.onChange(hostToPartitions, cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); + metadataState.onChange(hostToPartitions, Collections.emptyMap(), + cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostTwo, Utils.mkSet("global-table", "table-two", "table-one", "merged-table"), Utils.mkSet(topic2P0, topic1P1, topic2P2)); @@ -301,7 +305,7 @@ public void shouldGetMyMetadataForGlobalStoreWithKey() { @Test public void shouldGetAnyHostForGlobalStoreByKeyIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); - streamsMetadataState.onChange(hostToPartitions, cluster); + streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); assertNotNull(streamsMetadataState.getMetadataWithKey(globalTable, "key", Serdes.String().serializer())); } @@ -314,7 +318,7 @@ public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { @Test public void shouldGetAnyHostForGlobalStoreByKeyAndPartitionerIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); - streamsMetadataState.onChange(hostToPartitions, cluster); + streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); assertNotNull(streamsMetadataState.getMetadataWithKey(globalTable, "key", partitioner)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 8f313d5bece9f..0050ff5e63d50 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.Arrays; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; @@ -25,6 +26,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.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; @@ -952,8 +954,8 @@ public void testOnAssignment() { createMockTaskManager(); final Map> hostState = Collections.singletonMap( new HostInfo("localhost", 9090), - mkSet(t3p0, t3p3)); - taskManager.setPartitionsByHostState(hostState); + Utils.mkSet(t3p0, t3p3)); + taskManager.setHostPartitionMappings(hostState, Collections.emptyMap()); EasyMock.expectLastCall(); final Map> activeTasks = new HashMap<>(); @@ -973,7 +975,7 @@ public void testOnAssignment() { configurePartitionAssignor(emptyMap()); final List activeTaskList = asList(task0_0, task0_3); - final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTaskList, standbyTasks, hostState, 0); + final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTaskList, standbyTasks, hostState, Collections.emptyMap(), 0); final ConsumerPartitionAssignor.Assignment assignment = new ConsumerPartitionAssignor.Assignment(asList(t3p0, t3p3), info.encode()); partitionAssignor.onAssignment(assignment, null); @@ -1517,13 +1519,28 @@ public void shouldReturnInterleavedAssignmentWithUnrevokedPartitionsRemovedWhenN final AssignmentInfo decode = AssignmentInfo.decode(assignment.get(CONSUMER_1).userData()); assertThat( decode, - equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, asList(task0_0, task0_2), emptyMap(), emptyMap(), 0))); + equalTo(new AssignmentInfo( + LATEST_SUPPORTED_VERSION, + Arrays.asList(task0_0, task0_2), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + 0 + ))); + // The new consumer's assignment should be empty until c1 has the chance to revoke its partitions/tasks assertThat(assignment.get(CONSUMER_2).partitions(), equalTo(emptyList())); assertThat( AssignmentInfo.decode(assignment.get(CONSUMER_2).userData()), - equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, emptyList(), emptyMap(), emptyMap(), 0))); + equalTo(new AssignmentInfo( + LATEST_SUPPORTED_VERSION, + Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + 0 + ))); } @Test @@ -1573,12 +1590,12 @@ public void shouldReturnNormalAssignmentForOldAndFutureInstancesDuringVersionPro new ArrayList<>(activeTasks), standbyTaskMap, emptyMap(), + emptyMap(), 0 ) ) ); - assertThat(assignment.get("future-consumer").partitions(), equalTo(Collections.singletonList(t1p2))); assertThat( AssignmentInfo.decode(assignment.get("future-consumer").userData()), @@ -1588,6 +1605,7 @@ public void shouldReturnNormalAssignmentForOldAndFutureInstancesDuringVersionPro Collections.singletonList(task0_2), futureStandbyTaskMap, emptyMap(), + emptyMap(), 0) ) ); @@ -1625,13 +1643,13 @@ public void shouldReturnInterleavedAssignmentForOnlyFutureInstancesDuringVersion assertThat(assignment.get(CONSUMER_1).partitions(), equalTo(asList(t1p0, t1p2))); assertThat( AssignmentInfo.decode(assignment.get(CONSUMER_1).userData()), - equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, asList(task0_0, task0_2), emptyMap(), emptyMap(), 0))); + equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, asList(task0_0, task0_2), emptyMap(), emptyMap(), emptyMap(), 0))); assertThat(assignment.get(CONSUMER_2).partitions(), equalTo(Collections.singletonList(t1p1))); assertThat( AssignmentInfo.decode(assignment.get(CONSUMER_2).userData()), - equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, Collections.singletonList(task0_1), emptyMap(), emptyMap(), 0))); + equalTo(new AssignmentInfo(LATEST_SUPPORTED_VERSION, Collections.singletonList(task0_1), emptyMap(), emptyMap(), emptyMap(), 0))); } @Test @@ -1676,8 +1694,7 @@ private void shouldThrowIfPreVersionProbingSubscriptionAndFutureSubscriptionIsMi } private static ConsumerPartitionAssignor.Assignment createAssignment(final Map> firstHostState) { - final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, emptyList(), emptyMap(), firstHostState, 0); - + final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, emptyList(), emptyMap(), firstHostState, emptyMap(), 0); return new ConsumerPartitionAssignor.Assignment(emptyList(), info.encode()); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index 0d20454cd6f4d..774063809db3f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -55,52 +55,52 @@ public class AssignmentInfoTest { @Test public void shouldUseLatestSupportedVersionByDefault() { - final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, 0); + final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); assertEquals(LATEST_SUPPORTED_VERSION, info.version()); } @Test(expected = IllegalArgumentException.class) public void shouldThrowForUnknownVersion1() { - new AssignmentInfo(0, activeTasks, standbyTasks, globalAssignment, 0); + new AssignmentInfo(0, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); } @Test(expected = IllegalArgumentException.class) public void shouldThrowForUnknownVersion2() { - new AssignmentInfo(LATEST_SUPPORTED_VERSION + 1, activeTasks, standbyTasks, globalAssignment, 0); + new AssignmentInfo(LATEST_SUPPORTED_VERSION + 1, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); } @Test public void shouldEncodeAndDecodeVersion1() { - final AssignmentInfo info = new AssignmentInfo(1, activeTasks, standbyTasks, globalAssignment, 0); - final AssignmentInfo expectedInfo = new AssignmentInfo(1, UNKNOWN, activeTasks, standbyTasks, Collections.>emptyMap(), 0); + final AssignmentInfo info = new AssignmentInfo(1, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo expectedInfo = new AssignmentInfo(1, UNKNOWN, activeTasks, standbyTasks, Collections.emptyMap(), Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion2() { - final AssignmentInfo info = new AssignmentInfo(2, activeTasks, standbyTasks, globalAssignment, 0); - final AssignmentInfo expectedInfo = new AssignmentInfo(2, UNKNOWN, activeTasks, standbyTasks, globalAssignment, 0); + final AssignmentInfo info = new AssignmentInfo(2, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo expectedInfo = new AssignmentInfo(2, UNKNOWN, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion3() { - final AssignmentInfo info = new AssignmentInfo(3, activeTasks, standbyTasks, globalAssignment, 0); - final AssignmentInfo expectedInfo = new AssignmentInfo(3, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, 0); + final AssignmentInfo info = new AssignmentInfo(3, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo expectedInfo = new AssignmentInfo(3, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion4() { - final AssignmentInfo info = new AssignmentInfo(4, activeTasks, standbyTasks, globalAssignment, 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(4, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, 2); + final AssignmentInfo info = new AssignmentInfo(4, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(4, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion5() { - final AssignmentInfo info = new AssignmentInfo(5, activeTasks, standbyTasks, globalAssignment, 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(5, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, 2); + final AssignmentInfo info = new AssignmentInfo(5, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(5, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @@ -108,8 +108,8 @@ public void shouldEncodeAndDecodeVersion5() { public void shouldEncodeAndDecodeSmallerCommonlySupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; final int commonlySupportedVersion = LATEST_SUPPORTED_VERSION - 1; - final AssignmentInfo info = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, 2); + final AssignmentInfo info = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java index ce816195fb9db..19a903ce46b03 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProviderTest.java @@ -321,7 +321,7 @@ private StreamTask createStreamsTask(final StreamsConfig streamsConfig, } private void mockThread(final boolean initialized) { - EasyMock.expect(threadMock.isRunningAndNotRebalancing()).andReturn(initialized); + EasyMock.expect(threadMock.isRunning()).andReturn(initialized); EasyMock.expect(threadMock.tasks()).andStubReturn(tasks); EasyMock.replay(threadMock); } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java index 360f11e93e6d2..159db4eb1947a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.tests; +import java.util.Collections; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -225,7 +226,7 @@ public void onAssignment(final ConsumerPartitionAssignor.Assignment assignment, final TaskManager taskManager = taskManger(); taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo)); - taskManager.setPartitionsByHostState(partitionsByHost); + taskManager.setHostPartitionMappings(partitionsByHost, Collections.emptyMap()); taskManager.setPartitionsToTaskId(partitionsToTaskId); taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks()); taskManager.updateSubscriptionsFromAssignment(partitions); From 81392b1c2a0f346ec41f7283c93095dbe690b4cd Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Wed, 27 Nov 2019 15:52:54 +0530 Subject: [PATCH 02/21] Adding KeyQueryMetadata support for reads during rebalancing --- .../apache/kafka/streams/KafkaStreams.java | 44 ++-- .../processor/internals/KeyQueryMetadata.java | 93 ++++++++ .../internals/StreamsMetadataState.java | 208 +++++++++++------- .../internals/StreamsPartitionAssignor.java | 1 + .../internals/assignment/AssignmentInfo.java | 61 +++-- .../StreamsAssignmentProtocolVersions.java | 2 +- .../kafka/streams/state/StreamsMetadata.java | 18 +- .../common/message/SubscriptionInfo.json | 2 +- .../kafka/streams/KafkaStreamsTest.java | 4 +- .../QueryableStateIntegrationTest.java | 18 +- .../internals/StreamsMetadataStateTest.java | 61 +++-- .../assignment/AssignmentInfoTest.java | 7 + .../LegacySubscriptionInfoSerde.java | 4 +- .../assignment/SubscriptionInfoTest.java | 4 +- 14 files changed, 360 insertions(+), 167 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java 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 bbcb19058dc72..80743fefe7e55 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -46,14 +46,15 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.ThreadMetadata; -import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.GlobalStreamThread; -import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; -import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.StateDirectory; import org.apache.kafka.streams.processor.internals.StreamThread; import org.apache.kafka.streams.processor.internals.StreamsMetadataState; import org.apache.kafka.streams.processor.internals.ThreadStateTransitionValidator; +import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; +import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.QueryableStoreType; @@ -1020,7 +1021,7 @@ public void cleanUp() { */ public Collection allMetadata() { validateIsRunning(); - return streamsMetadataState.getAllActiveMetadata(); + return streamsMetadataState.getAllMetadata(); } /** @@ -1076,6 +1077,7 @@ public Collection allMetadataForStore(final String storeName) { * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@code storeName} and * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing */ + @Deprecated public StreamsMetadata metadataForKey(final String storeName, final K key, final Serializer keySerializer) { @@ -1107,6 +1109,7 @@ public StreamsMetadata metadataForKey(final String storeName, * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@code storeName} and * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing */ + @Deprecated public StreamsMetadata metadataForKey(final String storeName, final K key, final StreamPartitioner partitioner) { @@ -1115,22 +1118,31 @@ public StreamsMetadata metadataForKey(final String storeName, } /** - * [TENTATIVE] Provide metadata for all hosts that have this key. - * - * @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 List} for the key ordered active first and then standby or - * {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing + * Returns {@link KeyQueryMetadata} containing all metadata about hosting the given key for the given store. + */ + public KeyQueryMetadata queryMetadataForKey(final String storeName, + final K key, + final Serializer keySerializer) { + validateIsRunning(); + return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, keySerializer); + } + + /** + * Returns {@link KeyQueryMetadata} containing all metadata about hosting the given key for the given store, using the + * the supplied partitioner */ - public List allMetadataWithKey(final String storeName, - final K key, - final Serializer keySerializer) { + public KeyQueryMetadata queryMetadataForKey(final String storeName, + final K key, + final StreamPartitioner partitioner) { validateIsRunning(); - return streamsMetadataState.getAllMetadataWithKey(storeName, key, keySerializer); + return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, partitioner); } + /** + * Returns mapping from store name to another map of partition to offset lag info, for all stores local to this Streams instance. It includes both active and standby store partitions, with active partitions always reporting 0 lag. + */ + //public Map> allLocalOffsetLags() {} + /** * 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}. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java new file mode 100644 index 0000000000000..725b32eefb735 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java @@ -0,0 +1,93 @@ +/* + * 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.streams.state.HostInfo; + +import java.util.Collections; +import java.util.Set; + +public class KeyQueryMetadata { + /** + * Sentinel to indicate that the StreamsMetadata is currently unavailable. This can occur during rebalance + * operations. + */ + public final static KeyQueryMetadata NOT_AVAILABLE = new KeyQueryMetadata(new HostInfo("unavailable", -1), + Collections.emptySet(), + -1); + + // Active streams instance for key + private final HostInfo activeHost; + // Streams instances that host the key as standbys + private final Set standbyHosts; + + // Store partition corresponding to the key. + private final int partition; + + public KeyQueryMetadata(final HostInfo activeHost, final Set standbyHosts, final int partition) { + this.activeHost = activeHost; + this.standbyHosts = standbyHosts; + this.partition = partition; + } + + public HostInfo getActiveHost() { + return activeHost; + } + + public Set getStandbyHosts() { + return standbyHosts; + } + + public int getPartition() { + return partition; + } + + @Override + public boolean equals(final Object obj) { + if (this == obj) { + return true; + } + if (obj == null || getClass() != obj.getClass()) { + return false; + } + final KeyQueryMetadata that = (KeyQueryMetadata) obj; + if (activeHost != that.activeHost) { + return false; + } + if (!standbyHosts.equals(that.standbyHosts)) { + return false; + } + return partition == that.partition; + } + + @Override + public String toString() { + return "KeyQueryMetadata{" + + "activeHost=" + activeHost + + ", standbyHosts=" + standbyHosts + + ", partition=" + partition + + '}'; + } + + @Override + public int hashCode() { + int result = activeHost.hashCode(); + result = 31 * result + standbyHosts.hashCode(); + result = 31 * result + partition; + return result; + } +} 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 e3d01e17f2f9e..363c4ae0be184 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals; -import java.util.Arrays; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -45,8 +44,7 @@ public class StreamsMetadataState { public static final HostInfo UNKNOWN_HOST = new HostInfo("unknown", -1); private final InternalTopologyBuilder builder; - private final List allActiveMetadata = new ArrayList<>(); - private final List allStandbyMetadata = new ArrayList<>(); + private final List allMetadata = new ArrayList<>(); private final Set globalStores; private final HostInfo thisHost; private Cluster clusterMetadata; @@ -65,8 +63,7 @@ public String toString() { public String toString(final String indent) { final StringBuilder builder = new StringBuilder(); - builder.append(indent).append("GlobalActiveMetadata: ").append(allActiveMetadata).append("\n"); - builder.append(indent).append("GlobalStandbyMetadata: ").append(allStandbyMetadata).append("\n"); + builder.append(indent).append("GlobalMetadata: ").append(allMetadata).append("\n"); builder.append(indent).append("GlobalStores: ").append(globalStores).append("\n"); builder.append(indent).append("My HostInfo: ").append(thisHost).append("\n"); builder.append(indent).append(clusterMetadata).append("\n"); @@ -80,10 +77,10 @@ public String toString(final String indent) { * * @return all the {@link StreamsMetadata}s in a {@link KafkaStreams} application */ - public synchronized Collection getAllActiveMetadata() { - return allActiveMetadata; + public synchronized Collection getAllMetadata() { + return allMetadata; } - + /** * Find all of the {@link StreamsMetadata}s for a given storeName * @@ -98,7 +95,7 @@ public synchronized Collection getAllMetadataForStore(final Str } if (globalStores.contains(storeName)) { - return allActiveMetadata; + return allMetadata; } final List sourceTopics = builder.stateStoreNameToSourceTopics().get(storeName); @@ -107,8 +104,8 @@ public synchronized Collection getAllMetadataForStore(final Str } final ArrayList results = new ArrayList<>(); - for (final StreamsMetadata metadata : allActiveMetadata) { - if (metadata.stateStoreNames().contains(storeName)) { + for (final StreamsMetadata metadata : allMetadata) { + if (metadata.stateStoreNames().contains(storeName) || metadata.getStandbyStateStoreNames().contains(storeName)) { results.add(metadata); } } @@ -119,7 +116,7 @@ public synchronized Collection getAllMetadataForStore(final Str * Find the {@link StreamsMetadata}s for a given storeName and key. This method will use the * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used * please use {@link StreamsMetadataState#getMetadataWithKey(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 {@link StreamsMetadata} it would exist on. * @@ -130,6 +127,7 @@ public synchronized Collection getAllMetadataForStore(final Str * @return The {@link StreamsMetadata} for the storeName and key or {@link StreamsMetadata#NOT_AVAILABLE} * if streams is (re-)initializing */ + @Deprecated public synchronized StreamsMetadata getMetadataWithKey(final String storeName, final K key, final Serializer keySerializer) { @@ -145,7 +143,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return allActiveMetadata.get(0); + return allMetadata.get(0); } return myMetadata; } @@ -156,51 +154,100 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam } return getStreamsMetadataForKey(storeName, - key, - new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), - sourceTopicsInfo); + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), + sourceTopicsInfo); } /** - * TODO: rework based on final API design + * Find the {@link KeyQueryMetadata}s for a given storeName and key. + *

+ * Note: the key may not exist in the {@link StateStore}, + * this method provides a way of finding which {@link StreamsMetadata} it would exist on. + * + * @param storeName Name of the store + * @param key Key to use + * @param partitioner partitioner to use to find correct partition for key + * @param key type + * @return The {@link KeyQueryMetadata} for the storeName and key or {@link KeyQueryMetadata#NOT_AVAILABLE} + * if streams is (re-)initializing */ - public synchronized List getAllMetadataWithKey(final String storeName, - final K key, - final Serializer keySerializer) { - Objects.requireNonNull(keySerializer, "keySerializer can't be null"); + public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String storeName, + final K key, + final StreamPartitioner partitioner) { Objects.requireNonNull(storeName, "storeName can't be null"); Objects.requireNonNull(key, "key can't be null"); + Objects.requireNonNull(partitioner, "partitioner can't be null"); if (!isInitialized()) { - return Arrays.asList(StreamsMetadata.NOT_AVAILABLE); + return KeyQueryMetadata.NOT_AVAILABLE; } if (globalStores.contains(storeName)) { // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return allActiveMetadata; + return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), null, 0); } - return Arrays.asList(myMetadata); + return new KeyQueryMetadata(myMetadata.hostInfo(), null, 0); } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); if (sourceTopicsInfo == null) { return null; } - - return getAllStreamsMetadataForKey(storeName, - key, - new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), - sourceTopicsInfo); + return getKeyQueryMetadataForKey(storeName, key, partitioner, sourceTopicsInfo); } + /** + * Find the {@link KeyQueryMetadata}s for a given storeName and key. This method will use the + * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used + * please use {@link StreamsMetadataState#getKeyQueryMetadataWithKey(String, Object, Serializer)} + *

+ * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, + * this method provides a way of finding which {@link KeyQueryMetadata} it would exist on. + * + * @param storeName Name of the store + * @param key Key to use + * @param keySerializer Serializer for the key + * @param key type + * @return The {@link KeyQueryMetadata} for the storeName and key or {@link KeyQueryMetadata#NOT_AVAILABLE} + * if streams is (re-)initializing + */ + public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String storeName, + final K key, + final Serializer keySerializer) { + Objects.requireNonNull(keySerializer, "keySerializer can't be null"); + Objects.requireNonNull(storeName, "storeName can't be null"); + Objects.requireNonNull(key, "key can't be null"); + if (!isInitialized()) { + return KeyQueryMetadata.NOT_AVAILABLE; + } + if (globalStores.contains(storeName)) { + // global stores are on every node. if we dont' have the host info + // for this host then just pick the first metadata + if (thisHost == UNKNOWN_HOST) { + return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), null, 0); + } + return new KeyQueryMetadata(myMetadata.hostInfo(), null, 0); + } + + final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); + if (sourceTopicsInfo == null) { + return null; + } + + return getKeyQueryMetadataForKey(storeName, + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), + sourceTopicsInfo); + } /** * Find the {@link StreamsMetadata}s for a given storeName and key. - * + *

* Note: the key may not exist in the {@link StateStore}, * this method provides a way of finding which {@link StreamsMetadata} it would exist on. * @@ -211,6 +258,7 @@ public synchronized List getAllMetadataWithKey(final String * @return The {@link StreamsMetadata} for the storeName and key or {@link StreamsMetadata#NOT_AVAILABLE} * if streams is (re-)initializing */ + @Deprecated public synchronized StreamsMetadata getMetadataWithKey(final String storeName, final K key, final StreamPartitioner partitioner) { @@ -226,7 +274,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return allActiveMetadata.get(0); + return allMetadata.get(0); } return myMetadata; } @@ -241,12 +289,13 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam /** * Respond to changes to the HostInfo -> TopicPartition mapping. Will rebuild the * metadata - * @param activePartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for active partitions - * @param standbyPartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for standby partitions - * @param clusterMetadata the current clusterMetadata {@link Cluster} + * + * @param activePartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for active partitions + * @param standbyPartitionHostMap the current mapping of {@link HostInfo} -> {@link TopicPartition}s for standby partitions + * @param clusterMetadata the current clusterMetadata {@link Cluster} */ synchronized void onChange(final Map> activePartitionHostMap, - final Map> standbyPartitionHostMap, final Cluster clusterMetadata) { + final Map> standbyPartitionHostMap, final Cluster clusterMetadata) { this.clusterMetadata = clusterMetadata; rebuildMetadata(activePartitionHostMap, standbyPartitionHostMap); } @@ -273,36 +322,37 @@ private Set getStoresOnHost(final Map> stores, fina private void rebuildMetadata(final Map> activePartitionHostMap, - final Map> standbyPartitionHostMap) { - allActiveMetadata.clear(); - allStandbyMetadata.clear(); + final Map> standbyPartitionHostMap) { + allMetadata.clear(); if (activePartitionHostMap.isEmpty()) { return; } final Map> stores = builder.stateStoreNameToSourceTopics(); for (final Map.Entry> entry : activePartitionHostMap.entrySet()) { final HostInfo key = entry.getKey(); - final Set partitionsForHost = new HashSet<>(entry.getValue()); - final Set storesOnHost = getStoresOnHost(stores, partitionsForHost); - storesOnHost.addAll(globalStores); - final StreamsMetadata metadata = new StreamsMetadata(key, storesOnHost, partitionsForHost); - allActiveMetadata.add(metadata); + final Set activepartitionsForHost = new HashSet<>(entry.getValue()); + final Set activeStoresOnHost = getStoresOnHost(stores, activepartitionsForHost); + activeStoresOnHost.addAll(globalStores); + final Set standbypartitionsForHost = new HashSet<>(); + final Set standbyStoresOnHost = new HashSet<>(); + if (standbyPartitionHostMap != null && standbyPartitionHostMap.containsKey(key)) { + standbypartitionsForHost.addAll(standbyPartitionHostMap.get(key)); + standbyStoresOnHost.addAll(getStoresOnHost(stores, standbypartitionsForHost)); + } + final StreamsMetadata metadata = new StreamsMetadata(key, activeStoresOnHost, activepartitionsForHost, standbypartitionsForHost, standbyStoresOnHost); + if (activepartitionsForHost != null) { + allMetadata.add(metadata); + } if (key.equals(thisHost)) { myMetadata = metadata; } } - for (final Map.Entry> entry : standbyPartitionHostMap.entrySet()) { - final Set partitionsForHost = new HashSet<>(entry.getValue()); - final Set storesOnHost = getStoresOnHost(stores, partitionsForHost); - final StreamsMetadata metadata = new StreamsMetadata(entry.getKey(), storesOnHost, partitionsForHost); - allStandbyMetadata.add(metadata); - } } - private StreamsMetadata getStreamsMetadataForKey(final String storeName, - final K key, - final StreamPartitioner partitioner, - final SourceTopicsInfo sourceTopicsInfo) { + private KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, + final K key, + final StreamPartitioner partitioner, + final SourceTopicsInfo sourceTopicsInfo) { final Integer partition = partitioner.partition(sourceTopicsInfo.topicWithMostPartitions, key, null, sourceTopicsInfo.maxPartitions); final Set matchingPartitions = new HashSet<>(); @@ -310,36 +360,30 @@ private StreamsMetadata getStreamsMetadataForKey(final String storeName, matchingPartitions.add(new TopicPartition(sourceTopic, partition)); } - for (final StreamsMetadata streamsMetadata : allActiveMetadata) { - final Set stateStoreNames = streamsMetadata.stateStoreNames(); + HostInfo activeHost = new HostInfo("unavailable", -1); + final Set standbyHosts = new HashSet<>(); + for (final StreamsMetadata streamsMetadata : allMetadata) { + final Set activeStateStoreNames = streamsMetadata.stateStoreNames(); + final Set standbyStateStoreNames = streamsMetadata.getStandbyStateStoreNames(); final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); topicPartitions.retainAll(matchingPartitions); - if (stateStoreNames.contains(storeName) + if (activeStateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { - return streamsMetadata; + activeHost = streamsMetadata.hostInfo(); + } else if (standbyStateStoreNames.contains(storeName) + && !topicPartitions.isEmpty()) { + standbyHosts.add(streamsMetadata.hostInfo()); } } - return null; - } - private List getMatchingMetadata(final Set matchingPartitions, - final List allMetadata, final String storeName) { - final List matchingMetadata = new ArrayList<>(); - for (final StreamsMetadata streamsMetadata : allMetadata) { - final Set stateStoreNames = streamsMetadata.stateStoreNames(); - final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); - topicPartitions.retainAll(matchingPartitions); - if (stateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { - matchingMetadata.add(streamsMetadata); - } - } - return matchingMetadata; + return new KeyQueryMetadata(activeHost, standbyHosts, partition.intValue()); } - private List getAllStreamsMetadataForKey(final String storeName, - final K key, - final StreamPartitioner partitioner, - final SourceTopicsInfo sourceTopicsInfo) { + @Deprecated + private StreamsMetadata getStreamsMetadataForKey(final String storeName, + final K key, + final StreamPartitioner partitioner, + final SourceTopicsInfo sourceTopicsInfo) { final Integer partition = partitioner.partition(sourceTopicsInfo.topicWithMostPartitions, key, null, sourceTopicsInfo.maxPartitions); final Set matchingPartitions = new HashSet<>(); @@ -347,10 +391,16 @@ private List getAllStreamsMetadataForKey(final String store matchingPartitions.add(new TopicPartition(sourceTopic, partition)); } - final List matchingMetadata = new ArrayList<>(); - matchingMetadata.addAll(getMatchingMetadata(matchingPartitions, allActiveMetadata, storeName)); - matchingMetadata.addAll(getMatchingMetadata(matchingPartitions, allStandbyMetadata, storeName)); - return matchingMetadata; + for (final StreamsMetadata streamsMetadata : allMetadata) { + final Set stateStoreNames = streamsMetadata.stateStoreNames(); + final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); + topicPartitions.retainAll(matchingPartitions); + if (stateStoreNames.contains(storeName) + && !topicPartitions.isEmpty()) { + return streamsMetadata; + } + } + return null; } private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index 086396d9c454c..f69a33ef83afd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -1144,6 +1144,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat case 3: case 4: case 5: + case 6: processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); standbyPartitionsByHost = info.standbyPartitionByHost(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index 691afcb83eebb..0aa605721c87d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -55,12 +55,12 @@ public class AssignmentInfo { public AssignmentInfo(final int version, final int commonlySupportedVersion) { this(version, - commonlySupportedVersion, - Collections.emptyList(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), - 0); + commonlySupportedVersion, + Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + 0); } public AssignmentInfo(final int version, @@ -89,7 +89,7 @@ public AssignmentInfo(final int version, if (version < 1 || version > LATEST_SUPPORTED_VERSION) { throw new IllegalArgumentException("version must be between 1 and " + LATEST_SUPPORTED_VERSION - + "; was: " + version); + + "; was: " + version); } } @@ -123,7 +123,7 @@ public Map> standbyPartitionByHost() { /** * @throws TaskAssignmentException if method fails to encode the data, e.g., if there is an - * IO exception during encoding + * IO exception during encoding */ public ByteBuffer encode() { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -153,6 +153,13 @@ public ByteBuffer encode() { out.writeInt(errCode); break; case 5: + out.writeInt(usedVersion); + out.writeInt(commonlySupportedVersion); + encodeActiveAndStandbyTaskAssignment(out); + encodePartitionsByHostAsDictionary(out); + out.writeInt(errCode); + break; + case 6: out.writeInt(usedVersion); out.writeInt(commonlySupportedVersion); encodeActiveAndStandbyTaskAssignment(out); @@ -162,7 +169,7 @@ public ByteBuffer encode() { break; default: throw new IllegalStateException("Unknown metadata version: " + usedVersion - + "; latest commonly supported version: " + commonlySupportedVersion); + + "; latest commonly supported version: " + commonlySupportedVersion); } out.flush(); @@ -202,7 +209,7 @@ private void encodePartitionsByHost(final DataOutputStream out) throws IOExcepti } private void encodeHostPartitionMapAsDictionary(final DataOutputStream out, - final Map> hostPartitionMap) throws IOException { + final Map> hostPartitionMap) throws IOException { // Build a dictionary to encode topicNames int topicIndex = 0; @@ -299,6 +306,14 @@ public static AssignmentInfo decode(final ByteBuffer data) { assignmentInfo.errCode = in.readInt(); break; case 5: + commonlySupportedVersion = in.readInt(); + assignmentInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion); + decodeActiveTasks(assignmentInfo, in); + decodeStandbyTasks(assignmentInfo, in); + decodePartitionsByHostUsingDictionary(assignmentInfo, in); + assignmentInfo.errCode = in.readInt(); + break; + case 6: commonlySupportedVersion = in.readInt(); assignmentInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion); decodeActiveTasks(assignmentInfo, in); @@ -309,7 +324,7 @@ public static AssignmentInfo decode(final ByteBuffer data) { break; default: final TaskAssignmentException fatalException = new TaskAssignmentException("Unable to decode assignment data: " + - "used version: " + usedVersion + "; latest supported version: " + LATEST_SUPPORTED_VERSION); + "used version: " + usedVersion + "; latest supported version: " + LATEST_SUPPORTED_VERSION); log.error(fatalException.getMessage(), fatalException); throw fatalException; } @@ -340,7 +355,7 @@ private static void decodeStandbyTasks(final AssignmentInfo assignmentInfo, } private static void decodePartitionsByHost(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { + final DataInputStream in) throws IOException { assignmentInfo.partitionsByHost = new HashMap<>(); final int numEntries = in.readInt(); for (int i = 0; i < numEntries; i++) { @@ -359,7 +374,7 @@ private static Set readTopicPartitions(final DataInputStream in) } private static Map> decodeHostPartitionMapUsingDictionary(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { + final DataInputStream in) throws IOException { final Map> hostPartitionMap = new HashMap<>(); final int dictSize = in.readInt(); final Map topicIndexDict = new HashMap<>(dictSize); @@ -376,17 +391,17 @@ private static Map> decodeHostPartitionMapUsingDic } private static void decodePartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { + final DataInputStream in) throws IOException { assignmentInfo.partitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); } private static void decodeStandbyPartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { + final DataInputStream in) throws IOException { assignmentInfo.standbyPartitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); } private static Set readTopicPartitions(final DataInputStream in, - final Map topicIndexDict) throws IOException { + final Map topicIndexDict) throws IOException { final int numPartitions = in.readInt(); final Set partitions = new HashSet<>(numPartitions); for (int j = 0; j < numPartitions; j++) { @@ -399,7 +414,7 @@ private static Set readTopicPartitions(final DataInputStream in, public int hashCode() { final int hostMapHashCode = partitionsByHost.hashCode() ^ standbyPartitionsByHost.hashCode(); return usedVersion ^ commonlySupportedVersion ^ activeTasks.hashCode() ^ standbyTasks.hashCode() - ^ hostMapHashCode ^ errCode; + ^ hostMapHashCode ^ errCode; } @Override @@ -421,11 +436,11 @@ public boolean equals(final Object o) { @Override public String toString() { return "[version=" + usedVersion - + ", supported version=" + commonlySupportedVersion - + ", active tasks=" + activeTasks - + ", standby tasks=" + standbyTasks - + ", partitions by host=" + partitionsByHost - + ", standbyPartitions by host=" + standbyPartitionsByHost - + "]"; + + ", supported version=" + commonlySupportedVersion + + ", active tasks=" + activeTasks + + ", standby tasks=" + standbyTasks + + ", partitions by host=" + partitionsByHost + + ", standbyPartitions by host=" + standbyPartitionsByHost + + "]"; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java index ee9b7502cce30..f091907361644 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java @@ -19,7 +19,7 @@ public final class StreamsAssignmentProtocolVersions { public static final int UNKNOWN = -1; public static final int EARLIEST_PROBEABLE_VERSION = 3; - public static final int LATEST_SUPPORTED_VERSION = 5; + public static final int LATEST_SUPPORTED_VERSION = 6; private StreamsAssignmentProtocolVersions() {} } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index 16af13e54bc2a..1aecbed2f1769 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -35,20 +35,36 @@ public class StreamsMetadata { * operations. */ public final static StreamsMetadata NOT_AVAILABLE = new StreamsMetadata(new HostInfo("unavailable", -1), + Collections.emptySet(), + Collections.emptySet(), Collections.emptySet(), Collections.emptySet()); private final HostInfo hostInfo; private final Set stateStoreNames; private final Set topicPartitions; + private final Set standbyTopicPartitions; + private final Set standbyStateStoreNames; public StreamsMetadata(final HostInfo hostInfo, final Set stateStoreNames, - final Set topicPartitions) { + final Set topicPartitions, + final Set standbyTopicPartitions, + final Set standbyStateStoreNames) { this.hostInfo = hostInfo; this.stateStoreNames = stateStoreNames; this.topicPartitions = topicPartitions; + this.standbyTopicPartitions = standbyTopicPartitions; + this.standbyStateStoreNames = standbyStateStoreNames; + } + + public Set getStandbyTopicPartitions() { + return standbyTopicPartitions; + } + + public Set getStandbyStateStoreNames() { + return standbyStateStoreNames; } public HostInfo hostInfo() { diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index ff18f86459399..09e95feeb58f1 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -15,7 +15,7 @@ { "name": "SubscriptionInfo", - "validVersions": "1-5", + "validVersions": "1-6", "fields": [ { "name": "version", 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 cd24685165a07..5254970e744cb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -661,13 +661,13 @@ public void shouldNotGetAllTasksWithStoreWhenNotRunning() { @Test(expected = IllegalStateException.class) public void shouldNotGetTaskWithKeyAndSerializerWhenNotRunning() { final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); - streams.metadataForKey("store", "key", Serdes.String().serializer()); + streams.queryMetadataForKey("store", "key", Serdes.String().serializer()); } @Test(expected = IllegalStateException.class) public void shouldNotGetTaskWithKeyAndPartitionerWhenNotRunning() { final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); - streams.metadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); + streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); } @Test 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 70f646f882bc4..0fda4816bf44a 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 @@ -53,12 +53,12 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.streams.state.ReadOnlyWindowStore; -import org.apache.kafka.streams.state.StreamsMetadata; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockMapper; @@ -270,14 +270,14 @@ private void verifyAllKVKeys(final List streamsList, for (final String key: keys) { try { - final StreamsMetadata metadata = streams - .metadataForKey(storeName, key, new StringSerializer()); - if (metadata == null || metadata.equals(StreamsMetadata.NOT_AVAILABLE)) { + final KeyQueryMetadata metadata = streams + .queryMetadataForKey(storeName, key, new StringSerializer()); + if (metadata == null || metadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { noMetadataKeys.add(key); continue; } - final int index = metadata.hostInfo().port(); + final int index = metadata.getActiveHost().port(); final KafkaStreams streamsWithKey = streamsList.get(index); final ReadOnlyKeyValueStore store = streamsWithKey.store(storeName, QueryableStoreTypes.keyValueStore()); @@ -321,14 +321,14 @@ private void verifyAllWindowedKeys(final List streamsList, for (final String key: keys) { try { - final StreamsMetadata metadata = streams - .metadataForKey(storeName, key, new StringSerializer()); - if (metadata == null || metadata.equals(StreamsMetadata.NOT_AVAILABLE)) { + final KeyQueryMetadata metadata = streams + .queryMetadataForKey(storeName, key, new StringSerializer()); + if (metadata == null || metadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { noMetadataKeys.add(key); continue; } - final int index = metadata.hostInfo().port(); + final int index = metadata.getActiveHost().port(); final KafkaStreams streamsWithKey = streamsList.get(index); final ReadOnlyWindowStore store = streamsWithKey.store(storeName, QueryableStoreTypes.windowStore()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 9002c03284665..72cf57e98531a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -139,13 +139,13 @@ public void shouldNotThrowNPEWhenOnChangeNotCalled() { @Test public void shouldGetAllStreamInstances() { final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0)); + Utils.mkSet(topic1P0, topic2P1, topic4P0), null, null); final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1)); + Utils.mkSet(topic2P0, topic1P1), null, null); final StreamsMetadata three = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), - Collections.singleton(topic3P0)); + Collections.singleton(topic3P0), null, null); - final Collection actual = metadataState.getAllActiveMetadata(); + final Collection actual = metadataState.getAllMetadata(); assertEquals(3, actual.size()); assertTrue("expected " + actual + " to contain " + one, actual.contains(one)); assertTrue("expected " + actual + " to contain " + two, actual.contains(two)); @@ -169,17 +169,17 @@ public boolean test(final Object key, final Object value) { cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostFour, Collections.singleton(globalTable), - Collections.singleton(tp5)); - final Collection actual = metadataState.getAllActiveMetadata(); + Collections.singleton(tp5), null, null); + final Collection actual = metadataState.getAllMetadata(); assertTrue("expected " + actual + " to contain " + expected, actual.contains(expected)); } @Test public void shouldGetInstancesForStoreName() { final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0)); + Utils.mkSet(topic1P0, topic2P1, topic4P0), null, null); final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1)); + Utils.mkSet(topic2P0, topic1P1), null, null); final Collection actual = metadataState.getAllMetadataForStore("table-one"); assertEquals(2, actual.size()); assertTrue("expected " + actual + " to contain " + one, actual.contains(one)); @@ -205,10 +205,9 @@ public void shouldGetInstanceWithKey() { metadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); - final StreamsMetadata expected = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), - Collections.singleton(topic3P0)); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, Collections.emptySet(), 0); - final StreamsMetadata actual = metadataState.getMetadataWithKey("table-three", + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("table-three", "the-key", Serdes.String().serializer()); @@ -223,18 +222,19 @@ public void shouldGetInstanceWithKeyAndCustomPartitioner() { metadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); - final StreamsMetadata expected = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-three", "merged-table"), - Utils.mkSet(topic2P0, tp4)); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 1); - final StreamsMetadata actual = metadataState.getMetadataWithKey("table-three", "the-key", partitioner); + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("table-three", + "the-key", + partitioner); assertEquals(expected, actual); } @Test public void shouldReturnNotAvailableWhenClusterIsEmpty() { metadataState.onChange(Collections.>emptyMap(), Collections.emptyMap(), Cluster.empty()); - final StreamsMetadata result = metadataState.getMetadataWithKey("table-one", "a", Serdes.String().serializer()); - assertEquals(StreamsMetadata.NOT_AVAILABLE, result); + final KeyQueryMetadata result = metadataState.getKeyQueryMetadataWithKey("table-one", "a", Serdes.String().serializer()); + assertEquals(KeyQueryMetadata.NOT_AVAILABLE, result); } @Test @@ -242,12 +242,11 @@ public void shouldGetInstanceWithKeyWithMergedStreams() { final TopicPartition topic2P2 = new TopicPartition("topic-two", 2); hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1, topic2P2)); metadataState.onChange(hostToPartitions, Collections.emptyMap(), - cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); + cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); - final StreamsMetadata expected = new StreamsMetadata(hostTwo, Utils.mkSet("global-table", "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1, topic2P2)); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 2); - final StreamsMetadata actual = metadataState.getMetadataWithKey("merged-table", "123", new StreamPartitioner() { + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("merged-table", "the-key", new StreamPartitioner() { @Override public Integer partition(final String topic, final String key, final Object value, final int numPartitions) { return 2; @@ -260,7 +259,7 @@ public Integer partition(final String topic, final String key, final Object valu @Test public void shouldReturnNullOnGetWithKeyWhenStoreDoesntExist() { - final StreamsMetadata actual = metadataState.getMetadataWithKey("not-a-store", + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("not-a-store", "key", Serdes.String().serializer()); assertNull(actual); @@ -268,23 +267,23 @@ public void shouldReturnNullOnGetWithKeyWhenStoreDoesntExist() { @Test(expected = NullPointerException.class) public void shouldThrowWhenKeyIsNull() { - metadataState.getMetadataWithKey("table-three", null, Serdes.String().serializer()); + metadataState.getKeyQueryMetadataWithKey("table-three", null, Serdes.String().serializer()); } @Test(expected = NullPointerException.class) public void shouldThrowWhenSerializerIsNull() { - metadataState.getMetadataWithKey("table-three", "key", (Serializer) null); + metadataState.getKeyQueryMetadataWithKey("table-three", "key", (Serializer) null); } @Test(expected = NullPointerException.class) public void shouldThrowIfStoreNameIsNull() { - metadataState.getMetadataWithKey(null, "key", Serdes.String().serializer()); + metadataState.getKeyQueryMetadataWithKey(null, "key", Serdes.String().serializer()); } @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void shouldThrowIfStreamPartitionerIsNull() { - metadataState.getMetadataWithKey(null, "key", (StreamPartitioner) null); + metadataState.getKeyQueryMetadataWithKey(null, "key", (StreamPartitioner) null); } @Test @@ -298,28 +297,28 @@ public void shouldHaveGlobalStoreInAllMetadata() { @Test public void shouldGetMyMetadataForGlobalStoreWithKey() { - final StreamsMetadata metadata = metadataState.getMetadataWithKey(globalTable, "key", Serdes.String().serializer()); - assertEquals(hostOne, metadata.hostInfo()); + final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataWithKey(globalTable, "key", Serdes.String().serializer()); + assertEquals(hostOne, metadata.getActiveHost()); } @Test public void shouldGetAnyHostForGlobalStoreByKeyIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); - assertNotNull(streamsMetadataState.getMetadataWithKey(globalTable, "key", Serdes.String().serializer())); + assertNotNull(streamsMetadataState.getKeyQueryMetadataWithKey(globalTable, "key", Serdes.String().serializer())); } @Test public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { - final StreamsMetadata metadata = metadataState.getMetadataWithKey(globalTable, "key", partitioner); - assertEquals(hostOne, metadata.hostInfo()); + final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataWithKey(globalTable, "key", partitioner); + assertEquals(hostOne, metadata.getActiveHost()); } @Test public void shouldGetAnyHostForGlobalStoreByKeyAndPartitionerIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); - assertNotNull(streamsMetadataState.getMetadataWithKey(globalTable, "key", partitioner)); + assertNotNull(streamsMetadataState.getKeyQueryMetadataWithKey(globalTable, "key", partitioner)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index 774063809db3f..bc9a538d40251 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -104,6 +104,13 @@ public void shouldEncodeAndDecodeVersion5() { assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } + @Test + public void shouldEncodeAndDecodeVersion6() { + final AssignmentInfo info = new AssignmentInfo(6, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(6, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); + } + @Test public void shouldEncodeAndDecodeSmallerCommonlySupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java index fbfa78de63046..dd12cb859aa2d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/LegacySubscriptionInfoSerde.java @@ -32,7 +32,7 @@ public class LegacySubscriptionInfoSerde { private static final Logger log = LoggerFactory.getLogger(LegacySubscriptionInfoSerde.class); - public static final int LATEST_SUPPORTED_VERSION = 5; + public static final int LATEST_SUPPORTED_VERSION = 6; static final int UNKNOWN = -1; private final int usedVersion; @@ -95,7 +95,7 @@ public String userEndPoint() { * @throws TaskAssignmentException if method fails to encode the data */ public ByteBuffer encode() { - if (usedVersion == 3 || usedVersion == 4 || usedVersion == 5) { + if (usedVersion == 3 || usedVersion == 4 || usedVersion == 5 || usedVersion == 6) { final byte[] endPointBytes = prepareUserEndPoint(this.userEndPoint); final ByteBuffer buf = ByteBuffer.allocate( diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java index 227446c2dd42e..218ecbbb798b9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfoTest.java @@ -233,8 +233,8 @@ public void generatedVersion3And4ShouldBeDecodableByLegacyLogic() { } @Test - public void generatedVersion3To5ShouldDecodeLegacyFormat() { - for (int version = 3; version <= 5; version++) { + public void generatedVersion3To6ShouldDecodeLegacyFormat() { + for (int version = 3; version <= 6; version++) { final LegacySubscriptionInfoSerde info = new LegacySubscriptionInfoSerde( version, LATEST_SUPPORTED_VERSION, From 4a5f4e9fcfbb191059c07e4e7230edbd7abc696e Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Tue, 3 Dec 2019 20:39:10 +0530 Subject: [PATCH 03/21] Resolving code review comments --- .../apache/kafka/streams/KafkaStreams.java | 41 ++++++++++++------- .../processor/internals/KeyQueryMetadata.java | 31 +++++++------- .../internals/StreamsMetadataState.java | 23 +++++------ .../SuppressionIntegrationTest.java | 2 +- 4 files changed, 53 insertions(+), 44 deletions(-) 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 80743fefe7e55..96213f2d91969 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -213,7 +213,7 @@ public enum State { this.validTransitions.addAll(Arrays.asList(validTransitions)); } - public boolean isRunning() { + public boolean isRunningOrRebalancing() { return equals(RUNNING) || equals(REBALANCING); } @@ -297,17 +297,18 @@ public State state() { return state; } - private boolean isRunning() { + private boolean isRunningOrRebalancing() { synchronized (stateLock) { - return state.isRunning(); + return state.isRunningOrRebalancing(); } } - private void validateIsRunning() { - if (!isRunning()) { + private void validateIsRunningOrRebalancing() { + if (!isRunningOrRebalancing()) { throw new IllegalStateException("KafkaStreams is not running. State is " + state + "."); } } + /** * Listen to {@link State} change events. */ @@ -1004,7 +1005,7 @@ public synchronized boolean close(final Duration timeout) throws IllegalArgument * @throws StreamsException if cleanup failed */ public void cleanUp() { - if (isRunning()) { + if (isRunningOrRebalancing()) { throw new IllegalStateException("Cannot clean up while running."); } stateDirectory.clean(); @@ -1020,7 +1021,7 @@ public void cleanUp() { * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application */ public Collection allMetadata() { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getAllMetadata(); } @@ -1040,7 +1041,7 @@ public Collection allMetadata() { * this application */ public Collection allMetadataForStore(final String storeName) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getAllMetadataForStore(storeName); } @@ -1081,7 +1082,7 @@ public Collection allMetadataForStore(final String storeName) { public StreamsMetadata metadataForKey(final String storeName, final K key, final Serializer keySerializer) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getMetadataWithKey(storeName, key, keySerializer); } @@ -1113,28 +1114,40 @@ public StreamsMetadata metadataForKey(final String storeName, public StreamsMetadata metadataForKey(final String storeName, final K key, final StreamPartitioner partitioner) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getMetadataWithKey(storeName, key, partitioner); } /** + * Finds the metadata containing the active hosts and standby hosts where the key being queried would reside. + * + * @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 * Returns {@link KeyQueryMetadata} containing all metadata about hosting the given key for the given store. */ public KeyQueryMetadata queryMetadataForKey(final String storeName, final K key, final Serializer keySerializer) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, keySerializer); } /** + * Finds the metadata containing the active hosts and standby hosts where the key being queried would reside. + * + * @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 * Returns {@link KeyQueryMetadata} containing all metadata about hosting the given key for the given store, using the * the supplied partitioner */ public KeyQueryMetadata queryMetadataForKey(final String storeName, final K key, final StreamPartitioner partitioner) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, partitioner); } @@ -1156,7 +1169,7 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * {@code queryableStoreType} doesn't exist */ public T store(final String storeName, final QueryableStoreType queryableStoreType) { - validateIsRunning(); + validateIsRunningOrRebalancing(); return queryableStoreProvider.getStore(storeName, queryableStoreType); } @@ -1166,7 +1179,7 @@ public T store(final String storeName, final QueryableStoreType queryable * @return the set of {@link ThreadMetadata}. */ public Set localThreadsMetadata() { - validateIsRunning(); + validateIsRunningOrRebalancing(); final Set threadMetadata = new HashSet<>(); for (final StreamThread thread : threads) { threadMetadata.add(thread.threadMetadata()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java index 725b32eefb735..22eb8913e36a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java @@ -16,14 +16,23 @@ */ package org.apache.kafka.streams.processor.internals; +import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.state.HostInfo; import java.util.Collections; +import java.util.Objects; import java.util.Set; +/** + * Represents all the metadata where a particular key resides in a {@link KafkaStreams} application. + * It contains the user supplied {@link HostInfo} and the Set of standby Hosts where the key would be found in case an application + * has more than one standby. It also contains the partition number where the key belongs, this information would be useful in + * developing another apis on top, to fetch Offset Lag or Time based lag for the partition where the key belongs. + * NOTE: This is a point in time view. It may change when rebalances happen. + */ public class KeyQueryMetadata { /** - * Sentinel to indicate that the StreamsMetadata is currently unavailable. This can occur during rebalance + * Sentinel to indicate that the KeyQueryMetadata is currently unavailable. This can occur during rebalance * operations. */ public final static KeyQueryMetadata NOT_AVAILABLE = new KeyQueryMetadata(new HostInfo("unavailable", -1), @@ -58,20 +67,11 @@ public int getPartition() { @Override public boolean equals(final Object obj) { - if (this == obj) { - return true; - } - if (obj == null || getClass() != obj.getClass()) { - return false; - } - final KeyQueryMetadata that = (KeyQueryMetadata) obj; - if (activeHost != that.activeHost) { - return false; - } - if (!standbyHosts.equals(that.standbyHosts)) { + if (!(obj instanceof KeyQueryMetadata)) { return false; } - return partition == that.partition; + KeyQueryMetadata keyQueryMetadata = (KeyQueryMetadata) obj; + return Objects.equals(keyQueryMetadata.activeHost, activeHost) && Objects.equals(keyQueryMetadata.standbyHosts, standbyHosts) && Objects.equals(keyQueryMetadata.partition, partition); } @Override @@ -85,9 +85,6 @@ public String toString() { @Override public int hashCode() { - int result = activeHost.hashCode(); - result = 31 * result + standbyHosts.hashCode(); - result = 31 * result + partition; - return result; + return Objects.hash(activeHost, standbyHosts, partition); } } 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 363c4ae0be184..4bf6973e054c5 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 @@ -116,7 +116,6 @@ public synchronized Collection getAllMetadataForStore(final Str * Find the {@link StreamsMetadata}s for a given storeName and key. This method will use the * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used * please use {@link StreamsMetadataState#getMetadataWithKey(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 {@link StreamsMetadata} it would exist on. * @@ -161,7 +160,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam /** * Find the {@link KeyQueryMetadata}s for a given storeName and key. - *

+ * * Note: the key may not exist in the {@link StateStore}, * this method provides a way of finding which {@link StreamsMetadata} it would exist on. * @@ -187,9 +186,9 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), null, 0); + return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), Collections.emptySet(), -1); } - return new KeyQueryMetadata(myMetadata.hostInfo(), null, 0); + return new KeyQueryMetadata(myMetadata.hostInfo(), Collections.emptySet(), -1); } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); @@ -203,7 +202,7 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String * Find the {@link KeyQueryMetadata}s for a given storeName and key. This method will use the * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used * please use {@link StreamsMetadataState#getKeyQueryMetadataWithKey(String, Object, Serializer)} - *

+ * * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, * this method provides a way of finding which {@link KeyQueryMetadata} it would exist on. * @@ -229,9 +228,9 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String // global stores are on every node. if we dont' have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { - return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), null, 0); + return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), Collections.emptySet(), -1); } - return new KeyQueryMetadata(myMetadata.hostInfo(), null, 0); + return new KeyQueryMetadata(myMetadata.hostInfo(), Collections.emptySet(), -1); } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); @@ -333,13 +332,13 @@ private void rebuildMetadata(final Map> activePart final Set activepartitionsForHost = new HashSet<>(entry.getValue()); final Set activeStoresOnHost = getStoresOnHost(stores, activepartitionsForHost); activeStoresOnHost.addAll(globalStores); - final Set standbypartitionsForHost = new HashSet<>(); + final Set standbyPartitionsForHost = new HashSet<>(); final Set standbyStoresOnHost = new HashSet<>(); - if (standbyPartitionHostMap != null && standbyPartitionHostMap.containsKey(key)) { - standbypartitionsForHost.addAll(standbyPartitionHostMap.get(key)); - standbyStoresOnHost.addAll(getStoresOnHost(stores, standbypartitionsForHost)); + if (!standbyPartitionHostMap.isEmpty() && standbyPartitionHostMap.containsKey(key)) { + standbyPartitionsForHost.addAll(standbyPartitionHostMap.get(key)); + standbyStoresOnHost.addAll(getStoresOnHost(stores, standbyPartitionsForHost)); } - final StreamsMetadata metadata = new StreamsMetadata(key, activeStoresOnHost, activepartitionsForHost, standbypartitionsForHost, standbyStoresOnHost); + final StreamsMetadata metadata = new StreamsMetadata(key, activeStoresOnHost, activepartitionsForHost, standbyPartitionsForHost, standbyStoresOnHost); if (activepartitionsForHost != null) { allMetadata.add(metadata); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java index e3f0a4121e351..8e6b6acf20a7e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/SuppressionIntegrationTest.java @@ -343,7 +343,7 @@ private static void produceSynchronously(final String topic, final List !driver.state().isRunning(), DEFAULT_TIMEOUT, "Streams didn't shut down."); + waitForCondition(() -> !driver.state().isRunningOrRebalancing(), DEFAULT_TIMEOUT, "Streams didn't shut down."); assertThat(driver.state(), is(KafkaStreams.State.ERROR)); } } From d63ad2211330948b2ce311b8fd5c766b2e0cd336 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sat, 7 Dec 2019 22:49:58 +0530 Subject: [PATCH 04/21] Adding changes related to fetching offset lag on query --- .../org/apache/kafka/streams/KafkaStreams.java | 16 +++++++++++++++- .../processor/internals/AssignedTasks.java | 2 +- .../streams/processor/internals/StandbyTask.java | 5 +++++ .../streams/processor/internals/StreamTask.java | 7 +++++++ .../internals/StreamsMetadataState.java | 10 ++++++++++ .../StreamThreadStateStoreProvider.java | 2 ++ 6 files changed, 40 insertions(+), 2 deletions(-) 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 96213f2d91969..479df3db33b77 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -55,6 +55,9 @@ import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; +import org.apache.kafka.streams.processor.internals.StandbyTask; +import org.apache.kafka.streams.processor.internals.StreamTask; +import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.QueryableStoreType; @@ -1154,7 +1157,18 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, /** * Returns mapping from store name to another map of partition to offset lag info, for all stores local to this Streams instance. It includes both active and standby store partitions, with active partitions always reporting 0 lag. */ - //public Map> allLocalOffsetLags() {} + public Long storeLocalOffsetLag(final TaskId taskId, final TopicPartition topicPartition) { + for (int i = 0; i < this.threads.length; i++) { + final Map streamTaskMap = this.threads[i].tasks(); + final Map standyTaskMap = this.threads[i].standbyTasks(); + if (streamTaskMap.containsKey(taskId)) { + return streamTaskMap.get(taskId).offsetLimit(topicPartition); + } else if (standyTaskMap.containsKey(taskId)) { + return standyTaskMap.get(taskId).offsetLimit(topicPartition); + } + } + return -1L; + } /** * Get a facade wrapping the local {@link StateStore} instances with the provided {@code storeName} if the Store's diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java index 125d3e315e456..217548d297ceb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java @@ -251,7 +251,7 @@ int commit() { void shutdown(final boolean clean) { final AtomicReference firstException = new AtomicReference<>(null); - for (final T task: allTasks()) { + for (final T task : allTasks()) { try { closeTask(task, clean); } catch (final TaskMigratedException e) { 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 72ac76a97658b..6cfc2df4c98c1 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 @@ -221,6 +221,11 @@ private long updateOffsetLimits(final TopicPartition partition) { return offsetLimits.get(partition); } + public long offsetLimit(final TopicPartition partition) { + final Long limit = offsetLimits.get(partition); + return limit != null ? limit : Long.MAX_VALUE; + } + private Map committedOffsetForPartitions(final Set partitions) { try { // those do not have a committed offset would default to 0 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 a41a975c2e029..78c4cf51b8e0a 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 @@ -97,6 +97,7 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator private boolean transactionInFlight = false; private final String threadId; + private final Map offsetLimits = new HashMap<>(); public interface ProducerSupplier { Producer get(); @@ -245,6 +246,12 @@ private void initializeCommittedOffsets(final Map offsetsAndMetadata) { 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 4bf6973e054c5..9d7bce02fa532 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 @@ -23,6 +23,7 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StreamPartitioner; +import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.HostInfo; import org.apache.kafka.streams.state.StreamsMetadata; @@ -375,6 +376,15 @@ private KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, } } + // would need these below changes to send TopicGroupId instead of partition in KeyQueryMetadata + int myTopicGroupId; + final Map topicGroups = builder.topicGroups(); + for (Map.Entry topicGroup : topicGroups.entrySet()) { + if (topicGroup.getValue().sourceTopics.contains(sourceTopicsInfo.sourceTopics)) { + myTopicGroupId = topicGroup.getKey(); + } + } + return new KeyQueryMetadata(activeHost, standbyHosts, partition.intValue()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index 2bdc5a9c51da6..b9712a9b0a978 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -55,6 +55,8 @@ public List stores(final String storeName, final QueryableStoreType qu } final List stores = new ArrayList<>(); final Set tasks = new HashSet<>(streamThread.tasks().values()); + + //During rebalancing are standby tasks in running state? if (streamThread.standbyTasks() != null) { tasks.addAll(streamThread.standbyTasks().values()); } From 58d0f906ddae29b8dd882134688d12f318951e39 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sun, 8 Dec 2019 00:58:06 +0530 Subject: [PATCH 05/21] Reverting offset lag function to as discussed in KIP --- .../apache/kafka/streams/KafkaStreams.java | 33 ++++++++++++++----- .../processor/internals/AbstractTask.java | 2 ++ .../internals/InternalTopologyBuilder.java | 4 +++ .../processor/internals/StandbyTask.java | 1 + .../processor/internals/StreamTask.java | 1 + .../internals/StreamsMetadataState.java | 24 ++++++++------ .../processor/internals/AbstractTaskTest.java | 5 +++ 7 files changed, 52 insertions(+), 18 deletions(-) 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 479df3db33b77..103822f24600a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -57,6 +57,7 @@ import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; import org.apache.kafka.streams.processor.internals.StandbyTask; import org.apache.kafka.streams.processor.internals.StreamTask; +import org.apache.kafka.streams.processor.internals.AbstractTask; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.HostInfo; @@ -1155,19 +1156,35 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, } /** - * Returns mapping from store name to another map of partition to offset lag info, for all stores local to this Streams instance. It includes both active and standby store partitions, with active partitions always reporting 0 lag. + * Returns mapping from store name to another map of partition to offset lag info, for all stores local to this Streams instance. It includes both active and standby store partitions. */ - public Long storeLocalOffsetLag(final TaskId taskId, final TopicPartition topicPartition) { + public Map> allLocalOffsetLags() { + final Map> localOffsetLags = new HashMap<>(); + final Map> localOffsetLimits = new HashMap<>(); for (int i = 0; i < this.threads.length; i++) { final Map streamTaskMap = this.threads[i].tasks(); - final Map standyTaskMap = this.threads[i].standbyTasks(); - if (streamTaskMap.containsKey(taskId)) { - return streamTaskMap.get(taskId).offsetLimit(topicPartition); - } else if (standyTaskMap.containsKey(taskId)) { - return standyTaskMap.get(taskId).offsetLimit(topicPartition); + final Map standbyTaskMap = this.threads[i].standbyTasks(); + final Set activeStateStores = streamsMetadataState.getMyMetadata().stateStoreNames(); + final Set standbyStateStores = streamsMetadataState.getMyMetadata().getStandbyStateStoreNames(); + for (TaskId taskId : streamTaskMap.keySet()) { + localOffsetLimits.put(taskId, getStoreNameToOffsetMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); } + for (TaskId taskId : standbyTaskMap.keySet()) { + localOffsetLimits.put(taskId, getStoreNameToOffsetMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); + } + } + + //TODO: Need to calculate lags from diff(offset limit - current offsets) and recreate map with storeNames as key + return localOffsetLags; + } + + private Map getStoreNameToOffsetMap(final Set stateStores, final int partition, final AbstractTask task) { + final Map storeNameToOffsetMap = new HashMap<>(); + for (String storeName : stateStores) { + final TopicPartition topicPartition = new TopicPartition(streamsMetadataState.getChangelogTopicForStore(storeName), partition); + storeNameToOffsetMap.put(storeName, task.offsetLimit(topicPartition)); } - return -1L; + return storeNameToOffsetMap; } /** 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 a18b2b6a3590a..02bf521470e47 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 @@ -99,6 +99,8 @@ public TaskId id() { return id; } + public abstract long offsetLimit(final TopicPartition partition); + @Override public String applicationId() { return applicationId; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 5754b4a9f1ec2..11d7b0507ae6d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -599,6 +599,10 @@ public final void connectProcessorAndStateStores(final String processorName, nodeGroups = null; } + public Map getStoreToChangelogTopic() { + return storeToChangelogTopic; + } + public void connectSourceStoreAndTopic(final String sourceStoreName, final String topic) { if (storeToChangelogTopic.containsKey(sourceStoreName)) { 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 6cfc2df4c98c1..3859c3b3648b1 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 @@ -221,6 +221,7 @@ private long updateOffsetLimits(final TopicPartition partition) { return offsetLimits.get(partition); } + @Override public long offsetLimit(final TopicPartition partition) { final Long limit = offsetLimits.get(partition); return limit != null ? limit : Long.MAX_VALUE; 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 78c4cf51b8e0a..8d46189d39ece 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 @@ -249,6 +249,7 @@ private void initializeCommittedOffsets(final Map KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, } } - // would need these below changes to send TopicGroupId instead of partition in KeyQueryMetadata - int myTopicGroupId; - final Map topicGroups = builder.topicGroups(); - for (Map.Entry topicGroup : topicGroups.entrySet()) { - if (topicGroup.getValue().sourceTopics.contains(sourceTopicsInfo.sourceTopics)) { - myTopicGroupId = topicGroup.getKey(); - } - } - return new KeyQueryMetadata(activeHost, standbyHosts, partition.intValue()); } @@ -420,6 +420,10 @@ private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { return new SourceTopicsInfo(sourceTopics); } + public String getChangelogTopicForStore(final String storeName) { + return builder.getStoreToChangelogTopic().get(storeName); + } + private boolean isInitialized() { return clusterMetadata != null && !clusterMetadata.topics().isEmpty(); } 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 59573490358a5..b10abddc61633 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 @@ -218,6 +218,11 @@ private AbstractTask createTask(final Consumer consumer, stateDirectory, config) { + @Override + public long offsetLimit(TopicPartition partition) { + return 0; + } + @Override public void initializeMetadata() {} From a5f480605b75331efc89e0df00982f73545ab991 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sun, 8 Dec 2019 12:45:13 +0530 Subject: [PATCH 06/21] Adding support to fetch checkpointed offsets and calculate lag --- .../apache/kafka/streams/KafkaStreams.java | 32 +++++++++++-------- .../processor/internals/AbstractTask.java | 3 ++ .../processor/internals/StandbyTask.java | 3 +- .../processor/internals/StreamTask.java | 6 ++++ .../processor/internals/AbstractTaskTest.java | 5 +++ 5 files changed, 34 insertions(+), 15 deletions(-) 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 103822f24600a..cf0d7617572e2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1156,35 +1156,39 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, } /** - * Returns mapping from store name to another map of partition to offset lag info, for all stores local to this Streams instance. It includes both active and standby store partitions. + * Returns mapping from partition to another map of store name to offset lag info, for all {partitions, stores} local to this Streams instance. It includes both active and standby store partitions. */ - public Map> allLocalOffsetLags() { - final Map> localOffsetLags = new HashMap<>(); - final Map> localOffsetLimits = new HashMap<>(); + public Map> allLocalOffsetLags() { + final Map> localOffsetLags = new HashMap<>(); for (int i = 0; i < this.threads.length; i++) { final Map streamTaskMap = this.threads[i].tasks(); final Map standbyTaskMap = this.threads[i].standbyTasks(); final Set activeStateStores = streamsMetadataState.getMyMetadata().stateStoreNames(); final Set standbyStateStores = streamsMetadataState.getMyMetadata().getStandbyStateStoreNames(); - for (TaskId taskId : streamTaskMap.keySet()) { - localOffsetLimits.put(taskId, getStoreNameToOffsetMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); + for (final TaskId taskId : streamTaskMap.keySet()) { + localOffsetLags.put(taskId.partition, getStoreNameToLagMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); } - for (TaskId taskId : standbyTaskMap.keySet()) { - localOffsetLimits.put(taskId, getStoreNameToOffsetMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); + for (final TaskId taskId : standbyTaskMap.keySet()) { + localOffsetLags.put(taskId.partition, getStoreNameToLagMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); } } - //TODO: Need to calculate lags from diff(offset limit - current offsets) and recreate map with storeNames as key + // I feel returning Map> vs Map> are almost equivalent in this case and recreating + // Map> is more cumbersome, let me know if this makes sense. Also, even in folder structures, stores are + // present inside tasks and not vice verse. return localOffsetLags; } - private Map getStoreNameToOffsetMap(final Set stateStores, final int partition, final AbstractTask task) { - final Map storeNameToOffsetMap = new HashMap<>(); - for (String storeName : stateStores) { + private Map getStoreNameToLagMap(final Set stateStores, final int partition, final AbstractTask task) { + final Map storeNameToLagMap = new HashMap<>(); + for (final String storeName : stateStores) { final TopicPartition topicPartition = new TopicPartition(streamsMetadataState.getChangelogTopicForStore(storeName), partition); - storeNameToOffsetMap.put(storeName, task.offsetLimit(topicPartition)); + final long offsetLimit = task.offsetLimit(topicPartition); + final long checkpointedOffset = task.checkpointedOffsets().get(topicPartition); + final long offsetLag = offsetLimit - checkpointedOffset; + storeNameToLagMap.put(storeName, offsetLag); } - return storeNameToOffsetMap; + return storeNameToLagMap; } /** 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 02bf521470e47..8164e6e5a05d5 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 @@ -32,6 +32,7 @@ import java.io.IOException; import java.util.Collection; import java.util.HashSet; +import java.util.Map; import java.util.Set; public abstract class AbstractTask implements Task { @@ -101,6 +102,8 @@ public TaskId id() { public abstract long offsetLimit(final TopicPartition partition); + public abstract Map checkpointedOffsets(); + @Override public String applicationId() { return applicationId; 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 3859c3b3648b1..d6c1a1df58ba0 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 @@ -195,7 +195,8 @@ public List> update(final TopicPartition partitio return remainingRecords; } - Map checkpointedOffsets() { + @Override + public Map checkpointedOffsets() { return Collections.unmodifiableMap(stateMgr.checkpointed()); } 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 8d46189d39ece..b44c53bad1eba 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 @@ -57,6 +57,7 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; +import java.util.Collections; import java.util.stream.Collectors; import static java.lang.String.format; @@ -255,6 +256,11 @@ public long offsetLimit(final TopicPartition partition) { return limit != null ? limit : Long.MAX_VALUE; } + @Override + public Map checkpointedOffsets() { + return Collections.unmodifiableMap(stateMgr.checkpointed()); + } + private void initializeTaskTime(final Map offsetsAndMetadata) { for (final Map.Entry entry : offsetsAndMetadata.entrySet()) { final TopicPartition partition = entry.getKey(); 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 b10abddc61633..91af625f03efd 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 @@ -223,6 +223,11 @@ public long offsetLimit(TopicPartition partition) { return 0; } + @Override + public Map checkpointedOffsets() { + return null; + } + @Override public void initializeMetadata() {} From 9bdbad06f5b4bcc7985f601c98a0ce9b2508be2a Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sun, 8 Dec 2019 17:30:21 +0530 Subject: [PATCH 07/21] Minor checkstyle fixes and returning the planned offsetLag map --- .../apache/kafka/streams/KafkaStreams.java | 25 +++++++++++++------ .../processor/internals/KeyQueryMetadata.java | 4 +-- .../internals/StreamsMetadataState.java | 2 +- .../processor/internals/AbstractTaskTest.java | 2 +- 4 files changed, 21 insertions(+), 12 deletions(-) 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 cf0d7617572e2..dd4666e3ee4bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1158,25 +1158,34 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, /** * Returns mapping from partition to another map of store name to offset lag info, for all {partitions, stores} local to this Streams instance. It includes both active and standby store partitions. */ - public Map> allLocalOffsetLags() { - final Map> localOffsetLags = new HashMap<>(); + public Map> allLocalOffsetLags() { + final Map> localOffsetLags = new HashMap<>(); + final Map> finalLocalOffsetLags = new HashMap<>(); for (int i = 0; i < this.threads.length; i++) { final Map streamTaskMap = this.threads[i].tasks(); final Map standbyTaskMap = this.threads[i].standbyTasks(); final Set activeStateStores = streamsMetadataState.getMyMetadata().stateStoreNames(); final Set standbyStateStores = streamsMetadataState.getMyMetadata().getStandbyStateStoreNames(); for (final TaskId taskId : streamTaskMap.keySet()) { - localOffsetLags.put(taskId.partition, getStoreNameToLagMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); + localOffsetLags.put(taskId, getStoreNameToLagMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); } for (final TaskId taskId : standbyTaskMap.keySet()) { - localOffsetLags.put(taskId.partition, getStoreNameToLagMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); + localOffsetLags.put(taskId, getStoreNameToLagMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); } } - // I feel returning Map> vs Map> are almost equivalent in this case and recreating - // Map> is more cumbersome, let me know if this makes sense. Also, even in folder structures, stores are - // present inside tasks and not vice verse. - return localOffsetLags; + // Converting the Map> to Map> + for (final Map.Entry> entry : localOffsetLags.entrySet()) { + for (final Map.Entry storeToLagMap : entry.getValue().entrySet()) { + Map partitionToLagMap = finalLocalOffsetLags.get(storeToLagMap.getKey()); + if (partitionToLagMap == null) { + partitionToLagMap = new HashMap<>(); + } + partitionToLagMap.put(entry.getKey().partition, storeToLagMap.getValue()); + finalLocalOffsetLags.put(storeToLagMap.getKey(), partitionToLagMap); + } + } + return finalLocalOffsetLags; } private Map getStoreNameToLagMap(final Set stateStores, final int partition, final AbstractTask task) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java index 22eb8913e36a7..329f2863cc1bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java @@ -70,7 +70,7 @@ public boolean equals(final Object obj) { if (!(obj instanceof KeyQueryMetadata)) { return false; } - KeyQueryMetadata keyQueryMetadata = (KeyQueryMetadata) obj; + final KeyQueryMetadata keyQueryMetadata = (KeyQueryMetadata) obj; return Objects.equals(keyQueryMetadata.activeHost, activeHost) && Objects.equals(keyQueryMetadata.standbyHosts, standbyHosts) && Objects.equals(keyQueryMetadata.partition, partition); } @@ -85,6 +85,6 @@ public String toString() { @Override public int hashCode() { - return Objects.hash(activeHost, standbyHosts, partition); + return Objects.hash(activeHost, standbyHosts, partition); } } 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 0e939f7f1f02f..aef22b7f6e3fd 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 @@ -421,7 +421,7 @@ private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { } public String getChangelogTopicForStore(final String storeName) { - return builder.getStoreToChangelogTopic().get(storeName); + return builder.getStoreToChangelogTopic().get(storeName); } private boolean isInitialized() { 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 91af625f03efd..c8308dbf6196b 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 @@ -219,7 +219,7 @@ private AbstractTask createTask(final Consumer consumer, config) { @Override - public long offsetLimit(TopicPartition partition) { + public long offsetLimit(final TopicPartition partition) { return 0; } From 1268e757beefd8e4014217d58215c1470f79a07f Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 10 Dec 2019 16:38:45 -0800 Subject: [PATCH 08/21] Minor cleanups - Reimplemented KafkaStreams#allLocalOffsetLags() - Moved KeyQueryMetadata to org.apache.kafka.streams so we can import publicly - Minor renames, whitespace fixes --- .../apache/kafka/streams/KafkaStreams.java | 47 +++++++++---------- .../internals => }/KeyQueryMetadata.java | 3 +- .../processor/internals/AssignedTasks.java | 2 +- .../processor/internals/StandbyTask.java | 1 - .../internals/StreamsMetadataState.java | 13 +++-- .../kafka/streams/state/StreamsMetadata.java | 4 +- .../QueryableStateIntegrationTest.java | 2 +- .../internals/StreamsMetadataStateTest.java | 1 + 8 files changed, 34 insertions(+), 39 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/{processor/internals => }/KeyQueryMetadata.java (97%) 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 dd4666e3ee4bd..ef6c3b5dde6d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -54,7 +54,6 @@ import org.apache.kafka.streams.processor.internals.DefaultKafkaClientSupplier; import org.apache.kafka.streams.processor.internals.InternalTopologyBuilder; import org.apache.kafka.streams.processor.internals.ProcessorTopology; -import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; import org.apache.kafka.streams.processor.internals.StandbyTask; import org.apache.kafka.streams.processor.internals.StreamTask; import org.apache.kafka.streams.processor.internals.AbstractTask; @@ -1159,45 +1158,43 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, * Returns mapping from partition to another map of store name to offset lag info, for all {partitions, stores} local to this Streams instance. It includes both active and standby store partitions. */ public Map> allLocalOffsetLags() { - final Map> localOffsetLags = new HashMap<>(); - final Map> finalLocalOffsetLags = new HashMap<>(); + + final Map> localOffsetLags = new HashMap<>(); for (int i = 0; i < this.threads.length; i++) { final Map streamTaskMap = this.threads[i].tasks(); final Map standbyTaskMap = this.threads[i].standbyTasks(); final Set activeStateStores = streamsMetadataState.getMyMetadata().stateStoreNames(); - final Set standbyStateStores = streamsMetadataState.getMyMetadata().getStandbyStateStoreNames(); - for (final TaskId taskId : streamTaskMap.keySet()) { - localOffsetLags.put(taskId, getStoreNameToLagMap(activeStateStores, taskId.partition, streamTaskMap.get(taskId))); - } - for (final TaskId taskId : standbyTaskMap.keySet()) { - localOffsetLags.put(taskId, getStoreNameToLagMap(standbyStateStores, taskId.partition, standbyTaskMap.get(taskId))); - } - } + final Set standbyStateStores = streamsMetadataState.getMyMetadata().standbyStateStoreNames(); - // Converting the Map> to Map> - for (final Map.Entry> entry : localOffsetLags.entrySet()) { - for (final Map.Entry storeToLagMap : entry.getValue().entrySet()) { - Map partitionToLagMap = finalLocalOffsetLags.get(storeToLagMap.getKey()); - if (partitionToLagMap == null) { - partitionToLagMap = new HashMap<>(); - } - partitionToLagMap.put(entry.getKey().partition, storeToLagMap.getValue()); - finalLocalOffsetLags.put(storeToLagMap.getKey(), partitionToLagMap); + for (final Map.Entry taskEntry : streamTaskMap.entrySet()) { + addLagInfoToOutputMap(activeStateStores, taskEntry.getKey().partition, taskEntry.getValue(), localOffsetLags); + } + for (final Map.Entry taskEntry : standbyTaskMap.entrySet()) { + addLagInfoToOutputMap(standbyStateStores, taskEntry.getKey().partition, taskEntry.getValue(), localOffsetLags); } } - return finalLocalOffsetLags; + return localOffsetLags; } - private Map getStoreNameToLagMap(final Set stateStores, final int partition, final AbstractTask task) { - final Map storeNameToLagMap = new HashMap<>(); + private void addLagInfoToOutputMap(final Set stateStores, + final int partition, + final AbstractTask task, + final Map> outputLocalOffsetLags) { + for (final String storeName : stateStores) { final TopicPartition topicPartition = new TopicPartition(streamsMetadataState.getChangelogTopicForStore(storeName), partition); final long offsetLimit = task.offsetLimit(topicPartition); final long checkpointedOffset = task.checkpointedOffsets().get(topicPartition); final long offsetLag = offsetLimit - checkpointedOffset; - storeNameToLagMap.put(storeName, offsetLag); + + outputLocalOffsetLags.putIfAbsent(storeName, new HashMap<>()); + final Map partitionToOffsetLag = outputLocalOffsetLags.getOrDefault(storeName, new HashMap<>()); + if (!partitionToOffsetLag.containsKey(partition)) { + partitionToOffsetLag.put(partition, offsetLag); + } else { + throw new IllegalStateException("Encountered the same store partition" + storeName + "," + partition + " more than once"); + } } - return storeNameToLagMap; } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java similarity index 97% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java rename to streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java index 329f2863cc1bb..33b25e6dad593 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java @@ -14,9 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.kafka.streams.processor.internals; +package org.apache.kafka.streams; -import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.state.HostInfo; import java.util.Collections; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java index 217548d297ceb..125d3e315e456 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java @@ -251,7 +251,7 @@ int commit() { void shutdown(final boolean clean) { final AtomicReference firstException = new AtomicReference<>(null); - for (final T task : allTasks()) { + for (final T task: allTasks()) { try { closeTask(task, clean); } catch (final TaskMigratedException e) { 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 d6c1a1df58ba0..f785f837083ab 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 @@ -213,7 +213,6 @@ private long updateOffsetLimits(final TopicPartition partition) { throw new IllegalStateException("Offset limit should monotonically increase, but was reduced. " + "New limit: " + newlimit.getValue() + ". Previous limit: " + previousLimit); } - } offsetLimits.putAll(newLimits); 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 aef22b7f6e3fd..19f75db15bde5 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 @@ -21,6 +21,7 @@ import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.state.HostInfo; @@ -115,7 +116,7 @@ public synchronized Collection getAllMetadataForStore(final Str final ArrayList results = new ArrayList<>(); for (final StreamsMetadata metadata : allMetadata) { - if (metadata.stateStoreNames().contains(storeName) || metadata.getStandbyStateStoreNames().contains(storeName)) { + if (metadata.stateStoreNames().contains(storeName) || metadata.standbyStateStoreNames().contains(storeName)) { results.add(metadata); } } @@ -373,19 +374,17 @@ private KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, final Set standbyHosts = new HashSet<>(); for (final StreamsMetadata streamsMetadata : allMetadata) { final Set activeStateStoreNames = streamsMetadata.stateStoreNames(); - final Set standbyStateStoreNames = streamsMetadata.getStandbyStateStoreNames(); + final Set standbyStateStoreNames = streamsMetadata.standbyStateStoreNames(); final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); topicPartitions.retainAll(matchingPartitions); - if (activeStateStoreNames.contains(storeName) - && !topicPartitions.isEmpty()) { + if (activeStateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { activeHost = streamsMetadata.hostInfo(); - } else if (standbyStateStoreNames.contains(storeName) - && !topicPartitions.isEmpty()) { + } else if (standbyStateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { standbyHosts.add(streamsMetadata.hostInfo()); } } - return new KeyQueryMetadata(activeHost, standbyHosts, partition.intValue()); + return new KeyQueryMetadata(activeHost, standbyHosts, partition); } @Deprecated diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index 1aecbed2f1769..757033fef40e9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -59,11 +59,11 @@ public StreamsMetadata(final HostInfo hostInfo, this.standbyStateStoreNames = standbyStateStoreNames; } - public Set getStandbyTopicPartitions() { + public Set standbyTopicPartitions() { return standbyTopicPartitions; } - public Set getStandbyStateStoreNames() { + public Set standbyStateStoreNames() { return standbyStateStoreNames; } 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 0fda4816bf44a..a60d556bd16f9 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 @@ -53,7 +53,7 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.processor.internals.KeyQueryMetadata; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 72cf57e98531a..64754fcf759e0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -24,6 +24,7 @@ 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.KeyQueryMetadata; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyWrapper; From 4a3268bd79ae5f3fd9e4950120969a3479a43955 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 16 Dec 2019 21:33:17 -0800 Subject: [PATCH 09/21] Re-implement KafkaStreams#allLocalOffsetLags() by fetching endOffsets from Kafka - Re-implement the assignment protocol to serialize the topic index just once - Code cleanups, small bug fixes --- .../apache/kafka/streams/KafkaStreams.java | 106 ++++++++----- .../processor/internals/AbstractTask.java | 9 +- .../internals/AssignedStreamsTasks.java | 5 +- .../processor/internals/AssignedTasks.java | 2 +- .../internals/InternalTopologyBuilder.java | 9 ++ .../internals/ProcessorStateManager.java | 4 + .../processor/internals/StandbyTask.java | 7 - .../processor/internals/StreamTask.java | 12 -- .../processor/internals/StreamThread.java | 21 +-- .../internals/StreamsMetadataState.java | 147 +++++++++--------- .../internals/StreamsPartitionAssignor.java | 1 - .../processor/internals/TaskManager.java | 12 ++ .../internals/assignment/AssignmentInfo.java | 103 +++++++----- .../kafka/streams/state/StreamsMetadata.java | 31 +++- .../StreamThreadStateStoreProvider.java | 1 - .../processor/internals/AbstractTaskTest.java | 10 -- .../internals/StreamsMetadataStateTest.java | 38 ++--- 17 files changed, 290 insertions(+), 228 deletions(-) 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 ef6c3b5dde6d5..7d253c7bed17f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -16,7 +16,10 @@ */ package org.apache.kafka.streams; +import java.util.stream.Stream; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; +import org.apache.kafka.clients.admin.OffsetSpec; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; @@ -56,7 +59,6 @@ import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.StandbyTask; import org.apache.kafka.streams.processor.internals.StreamTask; -import org.apache.kafka.streams.processor.internals.AbstractTask; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl; import org.apache.kafka.streams.state.HostInfo; @@ -1134,7 +1136,7 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, final K key, final Serializer keySerializer) { validateIsRunningOrRebalancing(); - return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, keySerializer); + return streamsMetadataState.getKeyQueryMetadataForKey(storeName, key, keySerializer); } /** @@ -1151,50 +1153,84 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, final K key, final StreamPartitioner partitioner) { validateIsRunningOrRebalancing(); - return streamsMetadataState.getKeyQueryMetadataWithKey(storeName, key, partitioner); + return streamsMetadataState.getKeyQueryMetadataForKey(storeName, key, partitioner); } /** - * Returns mapping from partition to another map of store name to offset lag info, for all {partitions, stores} local to this Streams instance. It includes both active and standby store partitions. + * Returns offset lag info, for all store partitions (active or standby) local to this Streams instance. Note that the + * values returned are just estimates and meant to be used for making soft decisions on whether the data in the store + * partition is fresh enough for querying. + * + * @return map of store names to another map of partition to offset lags */ public Map> allLocalOffsetLags() { - final Map> localOffsetLags = new HashMap<>(); - for (int i = 0; i < this.threads.length; i++) { - final Map streamTaskMap = this.threads[i].tasks(); - final Map standbyTaskMap = this.threads[i].standbyTasks(); - final Set activeStateStores = streamsMetadataState.getMyMetadata().stateStoreNames(); - final Set standbyStateStores = streamsMetadataState.getMyMetadata().standbyStateStoreNames(); - - for (final Map.Entry taskEntry : streamTaskMap.entrySet()) { - addLagInfoToOutputMap(activeStateStores, taskEntry.getKey().partition, taskEntry.getValue(), localOffsetLags); + final Map standbyChangelogPositions = new HashMap<>(); + final Map activeChangelogPositions = new HashMap<>(); + final long unknownPosition = 0; + + // Obtain the current positions, of all the active-restoring and standby tasks + for (final StreamThread streamThread : this.threads) { + final Set restoringTaskIds = streamThread.restoringTaskIds(); + + for (final StandbyTask standbyTask : streamThread.allStandbyTasks()) { + final Map changelogPartitionLimits = standbyTask.checkpointedOffsets(); + standbyTask.changelogPartitions().forEach(topicPartition -> + standbyChangelogPositions.put(topicPartition, + changelogPartitionLimits.getOrDefault(topicPartition, unknownPosition))); } - for (final Map.Entry taskEntry : standbyTaskMap.entrySet()) { - addLagInfoToOutputMap(standbyStateStores, taskEntry.getKey().partition, taskEntry.getValue(), localOffsetLags); + + for (final StreamTask activeTask : streamThread.allStreamsTasks()) { + final boolean isRestoring = restoringTaskIds.contains(activeTask.id()); + final Map restoredOffsets = activeTask.restoredOffsets(); + activeTask.changelogPartitions().forEach(topicPartition -> { + if (isRestoring) { + activeChangelogPositions.put(topicPartition, restoredOffsets.getOrDefault(topicPartition, unknownPosition)); + } else { + activeChangelogPositions.put(topicPartition, unknownPosition); + } + }); } } - return localOffsetLags; - } - private void addLagInfoToOutputMap(final Set stateStores, - final int partition, - final AbstractTask task, - final Map> outputLocalOffsetLags) { - - for (final String storeName : stateStores) { - final TopicPartition topicPartition = new TopicPartition(streamsMetadataState.getChangelogTopicForStore(storeName), partition); - final long offsetLimit = task.offsetLimit(topicPartition); - final long checkpointedOffset = task.checkpointedOffsets().get(topicPartition); - final long offsetLag = offsetLimit - checkpointedOffset; - - outputLocalOffsetLags.putIfAbsent(storeName, new HashMap<>()); - final Map partitionToOffsetLag = outputLocalOffsetLags.getOrDefault(storeName, new HashMap<>()); - if (!partitionToOffsetLag.containsKey(partition)) { - partitionToOffsetLag.put(partition, offsetLag); - } else { - throw new IllegalStateException("Encountered the same store partition" + storeName + "," + partition + " more than once"); - } + log.info("Current changelog positions, for active: " + activeChangelogPositions + " standby:" + standbyChangelogPositions); + final Map offsetSpecMap = new HashMap<>(); + Stream.concat(activeChangelogPositions.keySet().stream(), standbyChangelogPositions.keySet().stream()) + .forEach(topicPartition -> offsetSpecMap.put(topicPartition, OffsetSpec.latest())); + try { + final Map allEndOffsets = adminClient.listOffsets(offsetSpecMap).all().get(); + log.info("Current end offsets :" + allEndOffsets); + allEndOffsets.forEach((topicPartition, offsetsResultInfo) -> { + final String storeName = streamsMetadataState.getStoreForChangelogTopic(topicPartition.topic()); + final long offsetPosition; + if (activeChangelogPositions.containsKey(topicPartition)) { + // if unknown, assume it's positioned at the tail of changelog partition + offsetPosition = activeChangelogPositions.get(topicPartition) == unknownPosition ? + offsetsResultInfo.offset() : activeChangelogPositions.get(topicPartition); + } else if (standbyChangelogPositions.containsKey(topicPartition)) { + // if unknown, assume it's positioned at the head of changelog partition + offsetPosition = standbyChangelogPositions.get(topicPartition) == unknownPosition ? + 0 : standbyChangelogPositions.get(topicPartition); + } else { + throw new IllegalStateException("Topic Partition " + topicPartition + " should be either active or standby"); + } + + final long offsetLag = offsetsResultInfo.offset() - offsetPosition; + final Map partitionToOffsetLag = localOffsetLags + .getOrDefault(storeName, new HashMap<>()); + if (!partitionToOffsetLag.containsKey(topicPartition.partition())) { + partitionToOffsetLag.put(topicPartition.partition(), offsetLag); + } else { + throw new IllegalStateException("Encountered the same store partition" + storeName + "," + + topicPartition.partition() + " more than once"); + } + localOffsetLags.put(storeName, partitionToOffsetLag); + }); + } catch (final Exception e) { + throw new StreamsException("Unable to obtain end offsets from kafka", e); } + + return localOffsetLags; } /** 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 8164e6e5a05d5..75317cc827851 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.Map; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; @@ -32,7 +33,6 @@ import java.io.IOException; import java.util.Collection; import java.util.HashSet; -import java.util.Map; import java.util.Set; public abstract class AbstractTask implements Task { @@ -100,10 +100,6 @@ public TaskId id() { return id; } - public abstract long offsetLimit(final TopicPartition partition); - - public abstract Map checkpointedOffsets(); - @Override public String applicationId() { return applicationId; @@ -260,4 +256,7 @@ public Collection changelogPartitions() { return stateMgr.changelogPartitions(); } + public Map restoredOffsets() { + return stateMgr.changelogReader().restoredOffsets(); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasks.java index cca6c3dc921f1..c198ecaec37ab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedStreamsTasks.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor.internals; import java.util.ArrayList; +import java.util.concurrent.ConcurrentHashMap; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.LogContext; @@ -34,8 +35,8 @@ import java.util.concurrent.atomic.AtomicReference; class AssignedStreamsTasks extends AssignedTasks implements RestoringTasks { - private final Map suspended = new HashMap<>(); - private final Map restoring = new HashMap<>(); + private final Map suspended = new ConcurrentHashMap<>(); + private final Map restoring = new ConcurrentHashMap<>(); private final Set restoredPartitions = new HashSet<>(); private final Map restoringByPartition = new HashMap<>(); private final Set prevActiveTasks = new HashSet<>(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java index 125d3e315e456..4d0c07e533150 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AssignedTasks.java @@ -39,7 +39,7 @@ abstract class AssignedTasks { final Logger log; final String taskTypeName; - final Map created = new HashMap<>(); + final Map created = new ConcurrentHashMap<>(); // IQ may access this map. final Map running = new ConcurrentHashMap<>(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java index 11d7b0507ae6d..24347fc2c4125 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopologyBuilder.java @@ -102,6 +102,9 @@ public class InternalTopologyBuilder { // map from state store names to this state store's corresponding changelog topic if possible private final Map storeToChangelogTopic = new HashMap<>(); + // map from changelog topic name to its corresponding state store. + private final Map changelogTopicToStore = new HashMap<>(); + // all global topics private final Set globalTopics = new HashSet<>(); @@ -603,12 +606,17 @@ public Map getStoreToChangelogTopic() { return storeToChangelogTopic; } + public Map getChangelogTopicToStore() { + return changelogTopicToStore; + } + public void connectSourceStoreAndTopic(final String sourceStoreName, final String topic) { if (storeToChangelogTopic.containsKey(sourceStoreName)) { throw new TopologyException("Source store " + sourceStoreName + " is already added."); } storeToChangelogTopic.put(sourceStoreName, topic); + changelogTopicToStore.put(topic, sourceStoreName); } public final void addInternalTopic(final String topicName) { @@ -944,6 +952,7 @@ private void buildProcessorNode(final Map processorMap, if (stateStoreFactory.loggingEnabled() && !storeToChangelogTopic.containsKey(stateStoreName)) { final String changelogTopic = ProcessorStateManager.storeChangelogTopic(applicationId, stateStoreName); storeToChangelogTopic.put(stateStoreName, changelogTopic); + changelogTopicToStore.put(changelogTopic, stateStoreName); } stateStoreMap.put(stateStoreName, stateStoreFactory.build()); } else { 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 9009a1c458fbf..87ee9607ad603 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 @@ -409,6 +409,10 @@ Collection changelogPartitions() { return unmodifiableList(changelogPartitions); } + ChangelogReader changelogReader() { + return changelogReader; + } + void ensureStoresRegistered() { for (final Map.Entry> entry : registeredStores.entrySet()) { if (!entry.getValue().isPresent()) { 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 f785f837083ab..4de3b5ea198c4 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 @@ -195,7 +195,6 @@ public List> update(final TopicPartition partitio return remainingRecords; } - @Override public Map checkpointedOffsets() { return Collections.unmodifiableMap(stateMgr.checkpointed()); } @@ -221,12 +220,6 @@ private long updateOffsetLimits(final TopicPartition partition) { return offsetLimits.get(partition); } - @Override - public long offsetLimit(final TopicPartition partition) { - final Long limit = offsetLimits.get(partition); - return limit != null ? limit : Long.MAX_VALUE; - } - private Map committedOffsetForPartitions(final Set partitions) { try { // those do not have a committed offset would default to 0 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 b44c53bad1eba..9a81526dc3d02 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 @@ -57,7 +57,6 @@ import java.util.HashSet; import java.util.Map; import java.util.Set; -import java.util.Collections; import java.util.stream.Collectors; import static java.lang.String.format; @@ -250,17 +249,6 @@ private void initializeCommittedOffsets(final Map checkpointedOffsets() { - return Collections.unmodifiableMap(stateMgr.checkpointed()); - } - private void initializeTaskTime(final Map offsetsAndMetadata) { for (final Map.Entry entry : offsetsAndMetadata.entrySet()) { final TopicPartition partition = entry.getKey(); 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 2902ef452f6a2..399dc0eda588c 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 @@ -235,11 +235,6 @@ State setState(final State newState) { return oldState; } - public boolean isRunningAndNotRebalancing() { - // we do not need to grab stateLock since it is a single read - return state == State.RUNNING; - } - public boolean isRunning() { synchronized (stateLock) { return state.isRunning(); @@ -1199,6 +1194,18 @@ public Map standbyTasks() { return taskManager.standbyTasks(); } + public List allStreamsTasks() { + return taskManager.allStreamsTasks(); + } + + public List allStandbyTasks() { + return taskManager.allStandbyTasks(); + } + + public Set restoringTaskIds() { + return taskManager.restoringTaskIds(); + } + /** * Produces a string representation containing useful information about a StreamThread. * This is useful in debugging scenarios. @@ -1269,8 +1276,4 @@ Map>> standbyRecords() { int currentNumIterations() { return numIterations; } - - public StreamThread.StateListener stateListener() { - return stateListener; - } } 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 19f75db15bde5..78d68c574d475 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.stream.Stream; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; @@ -91,7 +92,7 @@ public synchronized StreamsMetadata getMyMetadata() { public synchronized Collection getAllMetadata() { return allMetadata; } - + /** * Find all of the {@link StreamsMetadata}s for a given storeName * @@ -170,7 +171,31 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam } /** - * Find the {@link KeyQueryMetadata}s for a given storeName and key. + * Find the {@link KeyQueryMetadata}s for a given storeName and key. This method will use the + * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used + * please use {@link StreamsMetadataState#getKeyQueryMetadataForKey(String, Object, StreamPartitioner)} instead. + * + * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, + * this method provides a way of finding which {@link KeyQueryMetadata} it would exist on. + * + * @param storeName Name of the store + * @param key Key to use + * @param keySerializer Serializer for the key + * @param key type + * @return The {@link KeyQueryMetadata} for the storeName and key or {@link KeyQueryMetadata#NOT_AVAILABLE} + * if streams is (re-)initializing or null if the corresponding topic cannot be found + */ + public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, + final K key, + final Serializer keySerializer) { + Objects.requireNonNull(keySerializer, "keySerializer can't be null"); + return getKeyQueryMetadataForKey(storeName, + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata)); + } + + /** + * Find the {@link KeyQueryMetadata}s for a given storeName and key * * Note: the key may not exist in the {@link StateStore}, * this method provides a way of finding which {@link StreamsMetadata} it would exist on. @@ -182,7 +207,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam * @return The {@link KeyQueryMetadata} for the storeName and key or {@link KeyQueryMetadata#NOT_AVAILABLE} * if streams is (re-)initializing */ - public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String storeName, + public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, final K key, final StreamPartitioner partitioner) { Objects.requireNonNull(storeName, "storeName can't be null"); @@ -209,55 +234,9 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String return getKeyQueryMetadataForKey(storeName, key, partitioner, sourceTopicsInfo); } - /** - * Find the {@link KeyQueryMetadata}s for a given storeName and key. This method will use the - * {@link DefaultStreamPartitioner} to locate the store. If a custom partitioner has been used - * please use {@link StreamsMetadataState#getKeyQueryMetadataWithKey(String, Object, Serializer)} - * - * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, - * this method provides a way of finding which {@link KeyQueryMetadata} it would exist on. - * - * @param storeName Name of the store - * @param key Key to use - * @param keySerializer Serializer for the key - * @param key type - * @return The {@link KeyQueryMetadata} for the storeName and key or {@link KeyQueryMetadata#NOT_AVAILABLE} - * if streams is (re-)initializing - */ - public synchronized KeyQueryMetadata getKeyQueryMetadataWithKey(final String storeName, - final K key, - final Serializer keySerializer) { - Objects.requireNonNull(keySerializer, "keySerializer can't be null"); - Objects.requireNonNull(storeName, "storeName can't be null"); - Objects.requireNonNull(key, "key can't be null"); - - if (!isInitialized()) { - return KeyQueryMetadata.NOT_AVAILABLE; - } - - if (globalStores.contains(storeName)) { - // global stores are on every node. if we dont' have the host info - // for this host then just pick the first metadata - if (thisHost == UNKNOWN_HOST) { - return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), Collections.emptySet(), -1); - } - return new KeyQueryMetadata(myMetadata.hostInfo(), Collections.emptySet(), -1); - } - - final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); - if (sourceTopicsInfo == null) { - return null; - } - - return getKeyQueryMetadataForKey(storeName, - key, - new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), - sourceTopicsInfo); - } - /** * Find the {@link StreamsMetadata}s for a given storeName and key. - *

+ * * Note: the key may not exist in the {@link StateStore}, * this method provides a way of finding which {@link StreamsMetadata} it would exist on. * @@ -319,11 +298,11 @@ private boolean hasPartitionsForAnyTopics(final List topicNames, final S return false; } - private Set getStoresOnHost(final Map> stores, final Set partitionsForHost) { + private Set getStoresOnHost(final Map> storeToSourceTopics, final Set sourceTopicPartitions) { final Set storesOnHost = new HashSet<>(); - for (final Map.Entry> storeTopicEntry : stores.entrySet()) { + for (final Map.Entry> storeTopicEntry : storeToSourceTopics.entrySet()) { final List topicsForStore = storeTopicEntry.getValue(); - if (hasPartitionsForAnyTopics(topicsForStore, partitionsForHost)) { + if (hasPartitionsForAnyTopics(topicsForStore, sourceTopicPartitions)) { storesOnHost.add(storeTopicEntry.getKey()); } } @@ -334,29 +313,38 @@ private Set getStoresOnHost(final Map> stores, fina private void rebuildMetadata(final Map> activePartitionHostMap, final Map> standbyPartitionHostMap) { allMetadata.clear(); - if (activePartitionHostMap.isEmpty()) { + if (activePartitionHostMap.isEmpty() && standbyPartitionHostMap.isEmpty()) { return; } - final Map> stores = builder.stateStoreNameToSourceTopics(); - for (final Map.Entry> entry : activePartitionHostMap.entrySet()) { - final HostInfo key = entry.getKey(); - final Set activepartitionsForHost = new HashSet<>(entry.getValue()); - final Set activeStoresOnHost = getStoresOnHost(stores, activepartitionsForHost); - activeStoresOnHost.addAll(globalStores); - final Set standbyPartitionsForHost = new HashSet<>(); - final Set standbyStoresOnHost = new HashSet<>(); - if (!standbyPartitionHostMap.isEmpty() && standbyPartitionHostMap.containsKey(key)) { - standbyPartitionsForHost.addAll(standbyPartitionHostMap.get(key)); - standbyStoresOnHost.addAll(getStoresOnHost(stores, standbyPartitionsForHost)); - } - final StreamsMetadata metadata = new StreamsMetadata(key, activeStoresOnHost, activepartitionsForHost, standbyPartitionsForHost, standbyStoresOnHost); - if (activepartitionsForHost != null) { + final Map> storeToSourceTopics = builder.stateStoreNameToSourceTopics(); + Stream.concat(activePartitionHostMap.keySet().stream(), standbyPartitionHostMap.keySet().stream()) + .distinct() + .forEach(key -> { + final Set activePartitionsOnHost = new HashSet<>(); + final Set activeStoresOnHost = new HashSet<>(); + if (activePartitionHostMap.containsKey(key)) { + activePartitionsOnHost.addAll(activePartitionHostMap.get(key)); + activeStoresOnHost.addAll(getStoresOnHost(storeToSourceTopics, activePartitionsOnHost)); + } + activeStoresOnHost.addAll(globalStores); + + final Set standbyPartitionsOnHost = new HashSet<>(); + final Set standbyStoresOnHost = new HashSet<>(); + if (standbyPartitionHostMap.containsKey(key)) { + standbyPartitionsOnHost.addAll(standbyPartitionHostMap.get(key)); + standbyStoresOnHost.addAll(getStoresOnHost(storeToSourceTopics, standbyPartitionsOnHost)); + } + + final StreamsMetadata metadata = new StreamsMetadata(key, + activeStoresOnHost, + activePartitionsOnHost, + standbyPartitionsOnHost, + standbyStoresOnHost); allMetadata.add(metadata); - } - if (key.equals(thisHost)) { - myMetadata = metadata; - } - } + if (key.equals(thisHost)) { + myMetadata = metadata; + } + }); } private KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, @@ -374,12 +362,17 @@ private KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, final Set standbyHosts = new HashSet<>(); for (final StreamsMetadata streamsMetadata : allMetadata) { final Set activeStateStoreNames = streamsMetadata.stateStoreNames(); - final Set standbyStateStoreNames = streamsMetadata.standbyStateStoreNames(); final Set topicPartitions = new HashSet<>(streamsMetadata.topicPartitions()); + final Set standbyStateStoreNames = streamsMetadata.standbyStateStoreNames(); + final Set standbyTopicPartitions = new HashSet<>(streamsMetadata.standbyTopicPartitions()); + topicPartitions.retainAll(matchingPartitions); if (activeStateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { activeHost = streamsMetadata.hostInfo(); - } else if (standbyStateStoreNames.contains(storeName) && !topicPartitions.isEmpty()) { + } + + standbyTopicPartitions.retainAll(matchingPartitions); + if (standbyStateStoreNames.contains(storeName) && !standbyTopicPartitions.isEmpty()) { standbyHosts.add(streamsMetadata.hostInfo()); } } @@ -419,8 +412,8 @@ private SourceTopicsInfo getSourceTopicsInfo(final String storeName) { return new SourceTopicsInfo(sourceTopics); } - public String getChangelogTopicForStore(final String storeName) { - return builder.getStoreToChangelogTopic().get(storeName); + public String getStoreForChangelogTopic(final String topicName) { + return builder.getChangelogTopicToStore().get(topicName); } private boolean isInitialized() { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index f69a33ef83afd..f4e1d88e93c37 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -637,7 +637,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr topicPartitions.addAll(partitionsForTask.get(id)); } - // TODO: why do we have standby tasks even for non stateful sub topologies? for (final TaskId id : state.standbyTasks()) { standbyPartitions.addAll(partitionsForTask.get(id)); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index a2ddf9a6f181e..fd8ef181db8d1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -360,6 +360,18 @@ Map standbyTasks() { return standby.runningTaskMap(); } + List allStreamsTasks() { + return active.allTasks(); + } + + Set restoringTaskIds() { + return active.restoringTaskIds(); + } + + List allStandbyTasks() { + return standby.allTasks(); + } + void setConsumer(final Consumer consumer) { this.consumer = consumer; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index 0aa605721c87d..f43682f07abd0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -16,13 +16,8 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.ByteBufferInputStream; -import org.apache.kafka.streams.errors.TaskAssignmentException; -import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.state.HostInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -30,15 +25,22 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; - -import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; -import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.ByteBufferInputStream; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.HostInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class AssignmentInfo { private static final Logger log = LoggerFactory.getLogger(AssignmentInfo.class); @@ -163,8 +165,7 @@ public ByteBuffer encode() { out.writeInt(usedVersion); out.writeInt(commonlySupportedVersion); encodeActiveAndStandbyTaskAssignment(out); - encodePartitionsByHostAsDictionary(out); - encodeStandbyPartitionsByHostAsDictionary(out); + encodeActiveAndStandbyHostPartitions(out); out.writeInt(errCode); break; default: @@ -208,17 +209,31 @@ private void encodePartitionsByHost(final DataOutputStream out) throws IOExcepti } } - private void encodeHostPartitionMapAsDictionary(final DataOutputStream out, - final Map> hostPartitionMap) throws IOException { + private void encodeHostPartitionMapUsingDictionary(final DataOutputStream out, + final Map topicNameDict, + final Map> hostPartitionMap) throws IOException { + // encode partitions by host + out.writeInt(hostPartitionMap.size()); + + // Write the topic index, partition + for (final Map.Entry> entry : hostPartitionMap.entrySet()) { + writeHostInfo(out, entry.getKey()); + out.writeInt(entry.getValue().size()); + for (final TopicPartition partition : entry.getValue()) { + out.writeInt(topicNameDict.get(partition.topic())); + out.writeInt(partition.partition()); + } + } + } + private Map encodeTopicDictionaryAndGet(final DataOutputStream out, + final Set topicPartitions) throws IOException { // Build a dictionary to encode topicNames int topicIndex = 0; final Map topicNameDict = new HashMap<>(); - for (final Map.Entry> entry : hostPartitionMap.entrySet()) { - for (final TopicPartition topicPartition : entry.getValue()) { - if (!topicNameDict.containsKey(topicPartition.topic())) { - topicNameDict.put(topicPartition.topic(), topicIndex++); - } + for (final TopicPartition topicPartition : topicPartitions) { + if (!topicNameDict.containsKey(topicPartition.topic())) { + topicNameDict.put(topicPartition.topic(), topicIndex++); } } @@ -229,26 +244,23 @@ private void encodeHostPartitionMapAsDictionary(final DataOutputStream out, out.writeUTF(entry.getKey()); } - // encode partitions by host - out.writeInt(hostPartitionMap.size()); - - // Write the topic index, partition - for (final Map.Entry> entry : hostPartitionMap.entrySet()) { - writeHostInfo(out, entry.getKey()); - out.writeInt(entry.getValue().size()); - for (final TopicPartition partition : entry.getValue()) { - out.writeInt(topicNameDict.get(partition.topic())); - out.writeInt(partition.partition()); - } - } + return topicNameDict; } private void encodePartitionsByHostAsDictionary(final DataOutputStream out) throws IOException { - encodeHostPartitionMapAsDictionary(out, partitionsByHost); + final Set allTopicPartitions = partitionsByHost.values().stream() + .flatMap(Collection::stream).collect(Collectors.toSet()); + final Map topicNameDict = encodeTopicDictionaryAndGet(out, allTopicPartitions); + encodeHostPartitionMapUsingDictionary(out, topicNameDict, partitionsByHost); } - private void encodeStandbyPartitionsByHostAsDictionary(final DataOutputStream out) throws IOException { - encodeHostPartitionMapAsDictionary(out, standbyPartitionsByHost); + private void encodeActiveAndStandbyHostPartitions(final DataOutputStream out) throws IOException { + final Set allTopicPartitions = Stream + .concat(partitionsByHost.values().stream(), standbyPartitionsByHost.values().stream()) + .flatMap(Collection::stream).collect(Collectors.toSet()); + final Map topicNameDict = encodeTopicDictionaryAndGet(out, allTopicPartitions); + encodeHostPartitionMapUsingDictionary(out, topicNameDict, partitionsByHost); + encodeHostPartitionMapUsingDictionary(out, topicNameDict, standbyPartitionsByHost); } private void writeHostInfo(final DataOutputStream out, final HostInfo hostInfo) throws IOException { @@ -318,8 +330,7 @@ public static AssignmentInfo decode(final ByteBuffer data) { assignmentInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion); decodeActiveTasks(assignmentInfo, in); decodeStandbyTasks(assignmentInfo, in); - decodePartitionsByHostUsingDictionary(assignmentInfo, in); - decodeStandbyPartitionsByHostUsingDictionary(assignmentInfo, in); + decodeActiveAndStandbyHostPartitions(assignmentInfo, in); assignmentInfo.errCode = in.readInt(); break; default: @@ -373,15 +384,18 @@ private static Set readTopicPartitions(final DataInputStream in) return partitions; } - private static Map> decodeHostPartitionMapUsingDictionary(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { - final Map> hostPartitionMap = new HashMap<>(); + private static Map decodeTopicIndexAndGet(final DataInputStream in) throws IOException { final int dictSize = in.readInt(); final Map topicIndexDict = new HashMap<>(dictSize); for (int i = 0; i < dictSize; i++) { topicIndexDict.put(in.readInt(), in.readUTF()); } + return topicIndexDict; + } + private static Map> decodeHostPartitionMapUsingDictionary(final DataInputStream in, + final Map topicIndexDict) throws IOException { + final Map> hostPartitionMap = new HashMap<>(); final int numEntries = in.readInt(); for (int i = 0; i < numEntries; i++) { final HostInfo hostInfo = new HostInfo(in.readUTF(), in.readInt()); @@ -392,12 +406,15 @@ private static Map> decodeHostPartitionMapUsingDic private static void decodePartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, final DataInputStream in) throws IOException { - assignmentInfo.partitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); + final Map topicIndexDict = decodeTopicIndexAndGet(in); + assignmentInfo.partitionsByHost = decodeHostPartitionMapUsingDictionary(in, topicIndexDict); } - private static void decodeStandbyPartitionsByHostUsingDictionary(final AssignmentInfo assignmentInfo, - final DataInputStream in) throws IOException { - assignmentInfo.standbyPartitionsByHost = decodeHostPartitionMapUsingDictionary(assignmentInfo, in); + private static void decodeActiveAndStandbyHostPartitions(final AssignmentInfo assignmentInfo, + final DataInputStream in) throws IOException { + final Map topicIndexDict = decodeTopicIndexAndGet(in); + assignmentInfo.partitionsByHost = decodeHostPartitionMapUsingDictionary(in, topicIndexDict); + assignmentInfo.standbyPartitionsByHost = decodeHostPartitionMapUsingDictionary(in, topicIndexDict); } private static Set readTopicPartitions(final DataInputStream in, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index 757033fef40e9..e045beb50775d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.state; +import java.util.Objects; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.streams.KafkaStreams; @@ -41,10 +42,23 @@ public class StreamsMetadata { Collections.emptySet()); private final HostInfo hostInfo; + /** + * State stores owned by the instance as an active replica + */ private final Set stateStoreNames; + /** + * Topic partitions consumed by the instance as an active replica + */ private final Set topicPartitions; - private final Set standbyTopicPartitions; + /** + * State stores owned by the instance as a standby replica + */ private final Set standbyStateStoreNames; + /** + * (Source) Topic partitions for which the instance acts as standby. + */ + private final Set standbyTopicPartitions; + public StreamsMetadata(final HostInfo hostInfo, final Set stateStoreNames, @@ -103,16 +117,19 @@ public boolean equals(final Object o) { if (!stateStoreNames.equals(that.stateStoreNames)) { return false; } - return topicPartitions.equals(that.topicPartitions); + if (!topicPartitions.equals(that.topicPartitions)) { + return false; + } + if (!standbyStateStoreNames.equals(that.standbyStateStoreNames)) { + return false; + } + return standbyTopicPartitions.equals(that.standbyTopicPartitions); } @Override public int hashCode() { - int result = hostInfo.hashCode(); - result = 31 * result + stateStoreNames.hashCode(); - result = 31 * result + topicPartitions.hashCode(); - return result; + return Objects.hash(hostInfo, stateStoreNames, topicPartitions, standbyStateStoreNames, standbyTopicPartitions); } @Override @@ -121,6 +138,8 @@ public String toString() { "hostInfo=" + hostInfo + ", stateStoreNames=" + stateStoreNames + ", topicPartitions=" + topicPartitions + + ", standbyStateStoreNames=" + standbyStateStoreNames + + ", standbyTopicPartitions=" + standbyTopicPartitions + '}'; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index b9712a9b0a978..a7d43627e60db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -48,7 +48,6 @@ public List stores(final String storeName, final QueryableStoreType qu if (streamThread.state() == StreamThread.State.DEAD) { return Collections.emptyList(); } - // TODO: This needs to be rethought if (!streamThread.isRunning()) { throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the stream thread is " + streamThread.state() + ", not RUNNING"); 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 c8308dbf6196b..59573490358a5 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 @@ -218,16 +218,6 @@ private AbstractTask createTask(final Consumer consumer, stateDirectory, config) { - @Override - public long offsetLimit(final TopicPartition partition) { - return 0; - } - - @Override - public Map checkpointedOffsets() { - return null; - } - @Override public void initializeMetadata() {} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 64754fcf759e0..ebea8870dd936 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -140,11 +140,11 @@ public void shouldNotThrowNPEWhenOnChangeNotCalled() { @Test public void shouldGetAllStreamInstances() { final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), null, null); + Utils.mkSet(topic1P0, topic2P1, topic4P0), Collections.emptySet(), Collections.emptySet()); final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), null, null); + Utils.mkSet(topic2P0, topic1P1), Collections.emptySet(), Collections.emptySet()); final StreamsMetadata three = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), - Collections.singleton(topic3P0), null, null); + Collections.singleton(topic3P0), Collections.emptySet(), Collections.emptySet()); final Collection actual = metadataState.getAllMetadata(); assertEquals(3, actual.size()); @@ -170,7 +170,7 @@ public boolean test(final Object key, final Object value) { cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostFour, Collections.singleton(globalTable), - Collections.singleton(tp5), null, null); + Collections.singleton(tp5), Collections.emptySet(), Collections.emptySet()); final Collection actual = metadataState.getAllMetadata(); assertTrue("expected " + actual + " to contain " + expected, actual.contains(expected)); } @@ -178,9 +178,9 @@ public boolean test(final Object key, final Object value) { @Test public void shouldGetInstancesForStoreName() { final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), null, null); + Utils.mkSet(topic1P0, topic2P1, topic4P0), Collections.emptySet(), Collections.emptySet()); final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), null, null); + Utils.mkSet(topic2P0, topic1P1), Collections.emptySet(), Collections.emptySet()); final Collection actual = metadataState.getAllMetadataForStore("table-one"); assertEquals(2, actual.size()); assertTrue("expected " + actual + " to contain " + one, actual.contains(one)); @@ -208,7 +208,7 @@ public void shouldGetInstanceWithKey() { final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, Collections.emptySet(), 0); - final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("table-three", + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("table-three", "the-key", Serdes.String().serializer()); @@ -225,7 +225,7 @@ public void shouldGetInstanceWithKeyAndCustomPartitioner() { final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 1); - final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("table-three", + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("table-three", "the-key", partitioner); assertEquals(expected, actual); @@ -234,7 +234,7 @@ public void shouldGetInstanceWithKeyAndCustomPartitioner() { @Test public void shouldReturnNotAvailableWhenClusterIsEmpty() { metadataState.onChange(Collections.>emptyMap(), Collections.emptyMap(), Cluster.empty()); - final KeyQueryMetadata result = metadataState.getKeyQueryMetadataWithKey("table-one", "a", Serdes.String().serializer()); + final KeyQueryMetadata result = metadataState.getKeyQueryMetadataForKey("table-one", "a", Serdes.String().serializer()); assertEquals(KeyQueryMetadata.NOT_AVAILABLE, result); } @@ -247,7 +247,7 @@ public void shouldGetInstanceWithKeyWithMergedStreams() { final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 2); - final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("merged-table", "the-key", new StreamPartitioner() { + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("merged-table", "the-key", new StreamPartitioner() { @Override public Integer partition(final String topic, final String key, final Object value, final int numPartitions) { return 2; @@ -260,7 +260,7 @@ public Integer partition(final String topic, final String key, final Object valu @Test public void shouldReturnNullOnGetWithKeyWhenStoreDoesntExist() { - final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataWithKey("not-a-store", + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("not-a-store", "key", Serdes.String().serializer()); assertNull(actual); @@ -268,23 +268,23 @@ public void shouldReturnNullOnGetWithKeyWhenStoreDoesntExist() { @Test(expected = NullPointerException.class) public void shouldThrowWhenKeyIsNull() { - metadataState.getKeyQueryMetadataWithKey("table-three", null, Serdes.String().serializer()); + metadataState.getKeyQueryMetadataForKey("table-three", null, Serdes.String().serializer()); } @Test(expected = NullPointerException.class) public void shouldThrowWhenSerializerIsNull() { - metadataState.getKeyQueryMetadataWithKey("table-three", "key", (Serializer) null); + metadataState.getKeyQueryMetadataForKey("table-three", "key", (Serializer) null); } @Test(expected = NullPointerException.class) public void shouldThrowIfStoreNameIsNull() { - metadataState.getKeyQueryMetadataWithKey(null, "key", Serdes.String().serializer()); + metadataState.getKeyQueryMetadataForKey(null, "key", Serdes.String().serializer()); } @SuppressWarnings("unchecked") @Test(expected = NullPointerException.class) public void shouldThrowIfStreamPartitionerIsNull() { - metadataState.getKeyQueryMetadataWithKey(null, "key", (StreamPartitioner) null); + metadataState.getKeyQueryMetadataForKey(null, "key", (StreamPartitioner) null); } @Test @@ -298,7 +298,7 @@ public void shouldHaveGlobalStoreInAllMetadata() { @Test public void shouldGetMyMetadataForGlobalStoreWithKey() { - final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataWithKey(globalTable, "key", Serdes.String().serializer()); + final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataForKey(globalTable, "key", Serdes.String().serializer()); assertEquals(hostOne, metadata.getActiveHost()); } @@ -306,12 +306,12 @@ public void shouldGetMyMetadataForGlobalStoreWithKey() { public void shouldGetAnyHostForGlobalStoreByKeyIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); - assertNotNull(streamsMetadataState.getKeyQueryMetadataWithKey(globalTable, "key", Serdes.String().serializer())); + assertNotNull(streamsMetadataState.getKeyQueryMetadataForKey(globalTable, "key", Serdes.String().serializer())); } @Test public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { - final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataWithKey(globalTable, "key", partitioner); + final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataForKey(globalTable, "key", partitioner); assertEquals(hostOne, metadata.getActiveHost()); } @@ -319,7 +319,7 @@ public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { public void shouldGetAnyHostForGlobalStoreByKeyAndPartitionerIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); - assertNotNull(streamsMetadataState.getKeyQueryMetadataWithKey(globalTable, "key", partitioner)); + assertNotNull(streamsMetadataState.getKeyQueryMetadataForKey(globalTable, "key", partitioner)); } From 9b9a7e86d37ba3b75858b045c8b1a0c14f100e06 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 17 Dec 2019 14:07:24 -0800 Subject: [PATCH 10/21] Fix issue with reading standbyHostPartition maps for version <= 5 - Also cleanup non-essential code formatting changes --- .../apache/kafka/streams/KafkaStreams.java | 3 +- .../kafka/streams/KeyQueryMetadata.java | 11 +++-- .../processor/internals/StreamTask.java | 2 - .../internals/StreamsMetadataState.java | 24 +++++------ .../internals/StreamsPartitionAssignor.java | 4 ++ .../processor/internals/TaskManager.java | 2 +- .../internals/assignment/AssignmentInfo.java | 42 +++++++++---------- .../kafka/streams/state/StreamsMetadata.java | 2 +- .../StreamThreadStateStoreProvider.java | 1 - 9 files changed, 45 insertions(+), 46 deletions(-) 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 7d253c7bed17f..7099e73ced35a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1171,8 +1171,6 @@ public Map> allLocalOffsetLags() { // Obtain the current positions, of all the active-restoring and standby tasks for (final StreamThread streamThread : this.threads) { - final Set restoringTaskIds = streamThread.restoringTaskIds(); - for (final StandbyTask standbyTask : streamThread.allStandbyTasks()) { final Map changelogPartitionLimits = standbyTask.checkpointedOffsets(); standbyTask.changelogPartitions().forEach(topicPartition -> @@ -1180,6 +1178,7 @@ public Map> allLocalOffsetLags() { changelogPartitionLimits.getOrDefault(topicPartition, unknownPosition))); } + final Set restoringTaskIds = streamThread.restoringTaskIds(); for (final StreamTask activeTask : streamThread.allStreamsTasks()) { final boolean isRestoring = restoringTaskIds.contains(activeTask.id()); final Map restoredOffsets = activeTask.restoredOffsets(); diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java index 33b25e6dad593..8f723e43afed3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java @@ -23,11 +23,11 @@ import java.util.Set; /** - * Represents all the metadata where a particular key resides in a {@link KafkaStreams} application. - * It contains the user supplied {@link HostInfo} and the Set of standby Hosts where the key would be found in case an application - * has more than one standby. It also contains the partition number where the key belongs, this information would be useful in - * developing another apis on top, to fetch Offset Lag or Time based lag for the partition where the key belongs. - * NOTE: This is a point in time view. It may change when rebalances happen. + * Represents all the metadata related to a key, where a particular key resides in a {@link KafkaStreams} application. + * It contains the active {@link HostInfo} and a set of standby {@link HostInfo}s, denoting the instances where the key resides. + * It also contains the partition number where the key belongs, which could be useful when used in conjunction with other APIs. + * e.g: Relating with lags for that store partition using {@link KafkaStreams#allLocalOffsetLags()} + * NOTE: This is a point in time view. It may change as rebalances happen. */ public class KeyQueryMetadata { /** @@ -42,7 +42,6 @@ public class KeyQueryMetadata { private final HostInfo activeHost; // Streams instances that host the key as standbys private final Set standbyHosts; - // Store partition corresponding to the key. private final int partition; 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 9a81526dc3d02..a41a975c2e029 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 @@ -97,7 +97,6 @@ public class StreamTask extends AbstractTask implements ProcessorNodePunctuator private boolean transactionInFlight = false; private final String threadId; - private final Map offsetLimits = new HashMap<>(); public interface ProducerSupplier { Producer get(); @@ -246,7 +245,6 @@ private void initializeCommittedOffsets(final Map offsetsAndMetadata) { 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 78d68c574d475..89ad8a8547197 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 @@ -74,8 +74,7 @@ public String toString(final String indent) { } /** - * Get the {@link StreamsMetadata}s for local instance in a - * {@link KafkaStreams application} + * Get the {@link StreamsMetadata}s for the local instance in a {@link KafkaStreams application} * * @return the {@link StreamsMetadata}s for the local instance in a {@link KafkaStreams} application */ @@ -165,9 +164,9 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam } return getStreamsMetadataForKey(storeName, - key, - new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), - sourceTopicsInfo); + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata), + sourceTopicsInfo); } /** @@ -190,15 +189,15 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String final Serializer keySerializer) { Objects.requireNonNull(keySerializer, "keySerializer can't be null"); return getKeyQueryMetadataForKey(storeName, - key, - new DefaultStreamPartitioner<>(keySerializer, clusterMetadata)); + key, + new DefaultStreamPartitioner<>(keySerializer, clusterMetadata)); } /** * Find the {@link KeyQueryMetadata}s for a given storeName and key * - * Note: the key may not exist in the {@link StateStore}, - * this method provides a way of finding which {@link StreamsMetadata} it would exist on. + * Note: the key may not exist in the {@link StateStore},this method provides a way of finding which + * {@link StreamsMetadata} it would exist on. * * @param storeName Name of the store * @param key Key to use @@ -208,8 +207,8 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String * if streams is (re-)initializing */ public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String storeName, - final K key, - final StreamPartitioner partitioner) { + final K key, + final StreamPartitioner partitioner) { Objects.requireNonNull(storeName, "storeName can't be null"); Objects.requireNonNull(key, "key can't be null"); Objects.requireNonNull(partitioner, "partitioner can't be null"); @@ -284,7 +283,8 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam * @param clusterMetadata the current clusterMetadata {@link Cluster} */ synchronized void onChange(final Map> activePartitionHostMap, - final Map> standbyPartitionHostMap, final Cluster clusterMetadata) { + final Map> standbyPartitionHostMap, + final Cluster clusterMetadata) { this.clusterMetadata = clusterMetadata; rebuildMetadata(activePartitionHostMap, standbyPartitionHostMap); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java index f4e1d88e93c37..47edd3165a812 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java @@ -1143,6 +1143,10 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat case 3: case 4: case 5: + processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); + partitionsByHost = info.partitionsByHost(); + standbyPartitionsByHost = Collections.emptyMap(); + break; case 6: processVersionTwoAssignment(logPrefix, info, partitions, activeTasks, topicToPartitionInfo, partitionsToTaskId); partitionsByHost = info.partitionsByHost(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java index fd8ef181db8d1..8beeafa426140 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java @@ -453,7 +453,7 @@ public void setClusterMetadata(final Cluster cluster) { } public void setHostPartitionMappings(final Map> partitionsByHost, - final Map> standbyPartitionsByHost) { + final Map> standbyPartitionsByHost) { this.streamsMetadataState.onChange(partitionsByHost, standbyPartitionsByHost, cluster); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index f43682f07abd0..5cc2bf8ea4926 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -57,12 +57,12 @@ public class AssignmentInfo { public AssignmentInfo(final int version, final int commonlySupportedVersion) { this(version, - commonlySupportedVersion, - Collections.emptyList(), - Collections.emptyMap(), - Collections.emptyMap(), - Collections.emptyMap(), - 0); + commonlySupportedVersion, + Collections.emptyList(), + Collections.emptyMap(), + Collections.emptyMap(), + Collections.emptyMap(), + 0); } public AssignmentInfo(final int version, @@ -91,7 +91,7 @@ public AssignmentInfo(final int version, if (version < 1 || version > LATEST_SUPPORTED_VERSION) { throw new IllegalArgumentException("version must be between 1 and " + LATEST_SUPPORTED_VERSION - + "; was: " + version); + + "; was: " + version); } } @@ -125,7 +125,7 @@ public Map> standbyPartitionByHost() { /** * @throws TaskAssignmentException if method fails to encode the data, e.g., if there is an - * IO exception during encoding + * IO exception during encoding */ public ByteBuffer encode() { final ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -335,7 +335,7 @@ public static AssignmentInfo decode(final ByteBuffer data) { break; default: final TaskAssignmentException fatalException = new TaskAssignmentException("Unable to decode assignment data: " + - "used version: " + usedVersion + "; latest supported version: " + LATEST_SUPPORTED_VERSION); + "used version: " + usedVersion + "; latest supported version: " + LATEST_SUPPORTED_VERSION); log.error(fatalException.getMessage(), fatalException); throw fatalException; } @@ -439,12 +439,12 @@ public boolean equals(final Object o) { if (o instanceof AssignmentInfo) { final AssignmentInfo other = (AssignmentInfo) o; return usedVersion == other.usedVersion && - commonlySupportedVersion == other.commonlySupportedVersion && - errCode == other.errCode && - activeTasks.equals(other.activeTasks) && - standbyTasks.equals(other.standbyTasks) && - partitionsByHost.equals(other.partitionsByHost) && - standbyPartitionsByHost.equals(other.standbyPartitionsByHost); + commonlySupportedVersion == other.commonlySupportedVersion && + errCode == other.errCode && + activeTasks.equals(other.activeTasks) && + standbyTasks.equals(other.standbyTasks) && + partitionsByHost.equals(other.partitionsByHost) && + standbyPartitionsByHost.equals(other.standbyPartitionsByHost); } else { return false; } @@ -453,11 +453,11 @@ public boolean equals(final Object o) { @Override public String toString() { return "[version=" + usedVersion - + ", supported version=" + commonlySupportedVersion - + ", active tasks=" + activeTasks - + ", standby tasks=" + standbyTasks - + ", partitions by host=" + partitionsByHost - + ", standbyPartitions by host=" + standbyPartitionsByHost - + "]"; + + ", supported version=" + commonlySupportedVersion + + ", active tasks=" + activeTasks + + ", standby tasks=" + standbyTasks + + ", partitions by host=" + partitionsByHost + + ", standbyPartitions by host=" + standbyPartitionsByHost + + "]"; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index e045beb50775d..eca08a4e54514 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -134,7 +134,7 @@ public int hashCode() { @Override public String toString() { - return "StreamsMetadata{" + + return "StreamsMetadata {" + "hostInfo=" + hostInfo + ", stateStoreNames=" + stateStoreNames + ", topicPartitions=" + topicPartitions + diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index a7d43627e60db..478d9e37ec6a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -55,7 +55,6 @@ public List stores(final String storeName, final QueryableStoreType qu final List stores = new ArrayList<>(); final Set tasks = new HashSet<>(streamThread.tasks().values()); - //During rebalancing are standby tasks in running state? if (streamThread.standbyTasks() != null) { tasks.addAll(streamThread.standbyTasks().values()); } From 41c6dfabe48444115fb836abf588033fe759c5c8 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 23 Dec 2019 18:17:43 -0800 Subject: [PATCH 11/21] Adding tests for standby streams metadata --- .../internals/StreamsMetadataState.java | 4 +- .../internals/assignment/AssignmentInfo.java | 2 +- .../kafka/streams/state/StreamsMetadata.java | 4 +- .../kafka/streams/KafkaStreamsTest.java | 11 +- .../QueryableStateIntegrationTest.java | 182 ++++++++++++++---- .../internals/StreamsMetadataStateTest.java | 167 ++++++++-------- .../StreamsPartitionAssignorTest.java | 52 ++++- .../assignment/AssignmentInfoTest.java | 69 ++++--- .../streams/tests/StreamsUpgradeTest.java | 3 +- 9 files changed, 326 insertions(+), 168 deletions(-) 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 89ad8a8547197..1f1e3cbabc080 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 @@ -338,8 +338,8 @@ private void rebuildMetadata(final Map> activePart final StreamsMetadata metadata = new StreamsMetadata(key, activeStoresOnHost, activePartitionsOnHost, - standbyPartitionsOnHost, - standbyStoresOnHost); + standbyStoresOnHost, + standbyPartitionsOnHost); allMetadata.add(metadata); if (key.equals(thisHost)) { myMetadata = metadata; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index 5cc2bf8ea4926..5329e2cfd6d2f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -62,7 +62,7 @@ public AssignmentInfo(final int version, Collections.emptyMap(), Collections.emptyMap(), Collections.emptyMap(), - 0); + 0); } public AssignmentInfo(final int version, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index eca08a4e54514..0c6781cb50e63 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -63,8 +63,8 @@ public class StreamsMetadata { public StreamsMetadata(final HostInfo hostInfo, final Set stateStoreNames, final Set topicPartitions, - final Set standbyTopicPartitions, - final Set standbyStateStoreNames) { + final Set standbyStateStoreNames, + final Set standbyTopicPartitions) { this.hostInfo = hostInfo; this.stateStoreNames = stateStoreNames; 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 5254970e744cb..4cd427a78eeb2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -659,13 +659,20 @@ public void shouldNotGetAllTasksWithStoreWhenNotRunning() { } @Test(expected = IllegalStateException.class) - public void shouldNotGetTaskWithKeyAndSerializerWhenNotRunning() { + public void shouldNotGetQueryMetadataWithSerializerWhenNotRunningOrRebalancing() { final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); streams.queryMetadataForKey("store", "key", Serdes.String().serializer()); } + @Test + public void shouldGetQueryMetadataWithSerializerWhenRunningOrRebalancing() { + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); + streams.start(); + assertEquals(KeyQueryMetadata.NOT_AVAILABLE, streams.queryMetadataForKey("store", "key", Serdes.String().serializer())); + } + @Test(expected = IllegalStateException.class) - public void shouldNotGetTaskWithKeyAndPartitionerWhenNotRunning() { + public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing() { final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, supplier, time); streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); } 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 a60d556bd16f9..2148fe829e62f 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 @@ -16,13 +16,45 @@ */ package org.apache.kafka.streams.integration; +import static java.time.Duration.ofMillis; +import static java.time.Duration.ofSeconds; +import static java.time.Instant.ofEpochMilli; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; +import static org.apache.kafka.test.StreamsTestUtils.startKafkaStreamsAndWaitForRunningState; +import static org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +import java.io.BufferedReader; import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileReader; import java.io.IOException; import java.io.PrintStream; import java.io.StringReader; import java.time.Duration; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; +import java.util.Properties; +import java.util.Set; import java.util.TreeMap; +import java.util.TreeSet; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import kafka.utils.MockTime; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.KafkaProducer; @@ -38,6 +70,7 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KafkaStreams.State; import org.apache.kafka.streams.KafkaStreamsTest; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; @@ -53,7 +86,6 @@ import org.apache.kafka.streams.kstream.Produced; import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueMapper; -import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; @@ -68,44 +100,13 @@ import org.junit.After; import org.junit.Before; import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileReader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Properties; -import java.util.Set; -import java.util.TreeSet; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; - -import static java.time.Duration.ofMillis; -import static java.time.Duration.ofSeconds; -import static java.time.Instant.ofEpochMilli; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; -import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; -import static org.apache.kafka.test.TestUtils.retryOnExceptionWithTimeout; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.IsEqual.equalTo; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; -import static org.junit.Assert.fail; -import static org.apache.kafka.test.StreamsTestUtils.startKafkaStreamsAndWaitForRunningState; - @Category({IntegrationTest.class}) public class QueryableStateIntegrationTest { private static final Logger log = LoggerFactory.getLogger(QueryableStateIntegrationTest.class); @@ -114,6 +115,9 @@ public class QueryableStateIntegrationTest { private static final int NUM_BROKERS = 1; + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + @ClassRule public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); private static final int STREAM_THREE_PARTITIONS = 4; @@ -261,7 +265,8 @@ private void verifyAllKVKeys(final List streamsList, final KafkaStreamsTest.StateListenerStub stateListener, final Set keys, final String storeName, - final long timeout) throws Exception { + final long timeout, + final boolean pickInstanceByPort) throws Exception { retryOnExceptionWithTimeout(timeout, () -> { final List noMetadataKeys = new ArrayList<>(); final List nullStoreKeys = new ArrayList<>(); @@ -278,7 +283,7 @@ private void verifyAllKVKeys(final List streamsList, } final int index = metadata.getActiveHost().port(); - final KafkaStreams streamsWithKey = streamsList.get(index); + final KafkaStreams streamsWithKey = pickInstanceByPort ? streamsList.get(index) : streams; final ReadOnlyKeyValueStore store = streamsWithKey.store(storeName, QueryableStoreTypes.keyValueStore()); if (store == null) { @@ -312,7 +317,8 @@ private void verifyAllWindowedKeys(final List streamsList, final String storeName, final Long from, final Long to, - final long timeout) throws Exception { + final long timeout, + final boolean pickInstanceByPort) throws Exception { retryOnExceptionWithTimeout(timeout, () -> { final List noMetadataKeys = new ArrayList<>(); final List nullStoreKeys = new ArrayList<>(); @@ -329,7 +335,7 @@ private void verifyAllWindowedKeys(final List streamsList, } final int index = metadata.getActiveHost().port(); - final KafkaStreams streamsWithKey = streamsList.get(index); + final KafkaStreams streamsWithKey = pickInstanceByPort ? streamsList.get(index) : streams; final ReadOnlyWindowStore store = streamsWithKey.store(storeName, QueryableStoreTypes.windowStore()); if (store == null) { @@ -432,7 +438,8 @@ public void queryOnRebalance() throws Exception { listeners.get(i), inputValuesKeys, storeName + "-" + streamThree, - DEFAULT_TIMEOUT_MS); + DEFAULT_TIMEOUT_MS, + true); verifyAllWindowedKeys( streamsList, streamsList.get(i), @@ -441,7 +448,8 @@ public void queryOnRebalance() throws Exception { windowStoreName + "-" + streamThree, 0L, WINDOW_SIZE, - DEFAULT_TIMEOUT_MS); + DEFAULT_TIMEOUT_MS, + true); } // kill N-1 threads @@ -468,7 +476,96 @@ public void queryOnRebalance() throws Exception { listeners.get(0), inputValuesKeys, storeName + "-" + streamThree, - DEFAULT_TIMEOUT_MS); + DEFAULT_TIMEOUT_MS, + true); + verifyAllWindowedKeys( + streamsList, + streamsList.get(0), + listeners.get(0), + inputValuesKeys, + windowStoreName + "-" + streamThree, + 0L, + WINDOW_SIZE, + DEFAULT_TIMEOUT_MS, + true); + } finally { + for (final KafkaStreams streams : streamsList) { + streams.close(); + } + } + } + + @Test + public void shouldBeAbleQueryStandbyStateDuringRebalance() throws Exception { + final int numThreads = STREAM_TWO_PARTITIONS; + final List streamsList = new ArrayList<>(numThreads); + final List listeners = new ArrayList<>(numThreads); + + final ProducerRunnable producerRunnable = new ProducerRunnable(streamThree, inputValues, 1); + producerRunnable.run(); + + // create stream threads + final String storeName = "word-count-store"; + final String windowStoreName = "windowed-word-count-store"; + for (int i = 0; i < numThreads; i++) { + final Properties props = (Properties) streamsConfiguration.clone(); + final File stateDir = folder.newFolder("state-" + i); + props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + i); + props.put(StreamsConfig.CLIENT_ID_CONFIG, "instance-" + i); + props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.getAbsolutePath()); + final KafkaStreams streams = + createCountStream(streamThree, outputTopicThree, outputTopicConcurrentWindowed, storeName, windowStoreName, props); + final KafkaStreamsTest.StateListenerStub listener = new KafkaStreamsTest.StateListenerStub(); + streams.setStateListener(listener); + listeners.add(listener); + streamsList.add(streams); + } + startApplicationAndWaitUntilRunning(streamsList, Duration.ofSeconds(60)); + + try { + waitUntilAtLeastNumRecordProcessed(outputTopicThree, 1); + + // Ensure each thread can serve all keys by itself; i.e standby replication works. + for (int i = 0; i < streamsList.size(); i++) { + verifyAllKVKeys( + streamsList, + streamsList.get(i), + listeners.get(i), + inputValuesKeys, + storeName + "-" + streamThree, + DEFAULT_TIMEOUT_MS, + false); + verifyAllWindowedKeys( + streamsList, + streamsList.get(i), + listeners.get(i), + inputValuesKeys, + windowStoreName + "-" + streamThree, + 0L, + WINDOW_SIZE, + DEFAULT_TIMEOUT_MS, + false); + } + + // kill N-1 threads + for (int i = 1; i < streamsList.size(); i++) { + final Duration closeTimeout = Duration.ofSeconds(60); + assertThat(String.format("Streams instance %s did not close in %d ms", i, closeTimeout.toMillis()), + streamsList.get(i).close(closeTimeout)); + } + + waitForApplicationState(streamsList.subList(1, numThreads), State.NOT_RUNNING, Duration.ofSeconds(60)); + + // Now, confirm that all the keys are still queryable on the remaining thread, regardless of the state + verifyAllKVKeys( + streamsList, + streamsList.get(0), + listeners.get(0), + inputValuesKeys, + storeName + "-" + streamThree, + DEFAULT_TIMEOUT_MS, + false); verifyAllWindowedKeys( streamsList, streamsList.get(0), @@ -477,7 +574,8 @@ public void queryOnRebalance() throws Exception { windowStoreName + "-" + streamThree, 0L, WINDOW_SIZE, - DEFAULT_TIMEOUT_MS); + DEFAULT_TIMEOUT_MS, + false); } finally { for (final KafkaStreams streams : streamsList) { streams.close(); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index ebea8870dd936..6fd18172c182f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -16,42 +16,39 @@ */ package org.apache.kafka.streams.processor.internals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; 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.KeyQueryMetadata; -import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyWrapper; +import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.state.HostInfo; -import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StreamsMetadata; import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - public class StreamsMetadataStateTest { private StreamsMetadataState metadataState; @@ -61,7 +58,8 @@ public class StreamsMetadataStateTest { private TopicPartition topic1P0; private TopicPartition topic2P0; private TopicPartition topic3P0; - private Map> hostToPartitions; + private Map> hostToActivePartitions; + private Map> hostToStandbyPartitions; private StreamsBuilder builder; private TopicPartition topic1P1; private TopicPartition topic2P1; @@ -74,27 +72,22 @@ public class StreamsMetadataStateTest { public void before() { builder = new StreamsBuilder(); final KStream one = builder.stream("topic-one"); - one.groupByKey().count(Materialized.>as("table-one")); + one.groupByKey().count(Materialized.as("table-one")); final KStream two = builder.stream("topic-two"); - two.groupByKey().count(Materialized.>as("table-two")); + two.groupByKey().count(Materialized.as("table-two")); builder.stream("topic-three") .groupByKey() - .count(Materialized.>as("table-three")); + .count(Materialized.as("table-three")); - one.merge(two).groupByKey().count(Materialized.>as("merged-table")); + one.merge(two).groupByKey().count(Materialized.as("merged-table")); - builder.stream("topic-four").mapValues(new ValueMapper() { - @Override - public Object apply(final Object value) { - return value; - } - }); + builder.stream("topic-four").mapValues(value -> value); builder.globalTable("global-topic", Consumed.with(null, null), - Materialized.>as(globalTable)); + Materialized.as(globalTable)); TopologyWrapper.getInternalTopologyBuilder(builder.build()).setApplicationId("appId"); @@ -108,10 +101,14 @@ public Object apply(final Object value) { hostOne = new HostInfo("host-one", 8080); hostTwo = new HostInfo("host-two", 9090); hostThree = new HostInfo("host-three", 7070); - hostToPartitions = new HashMap<>(); - hostToPartitions.put(hostOne, Utils.mkSet(topic1P0, topic2P1, topic4P0)); - hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1)); - hostToPartitions.put(hostThree, Collections.singleton(topic3P0)); + hostToActivePartitions = new HashMap<>(); + hostToActivePartitions.put(hostOne, Utils.mkSet(topic1P0, topic2P1, topic4P0)); + hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1)); + hostToActivePartitions.put(hostThree, Collections.singleton(topic3P0)); + hostToStandbyPartitions = new HashMap<>(); + hostToStandbyPartitions.put(hostThree, Utils.mkSet(topic1P0, topic2P1, topic4P0)); + hostToStandbyPartitions.put(hostOne, Utils.mkSet(topic2P0, topic1P1)); + hostToStandbyPartitions.put(hostTwo, Collections.singleton(topic3P0)); final List partitionInfos = Arrays.asList( new PartitionInfo("topic-one", 0, null, null, null), @@ -123,28 +120,32 @@ public Object apply(final Object value) { cluster = new Cluster(null, Collections.emptyList(), partitionInfos, Collections.emptySet(), Collections.emptySet()); metadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne); - metadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); - partitioner = new StreamPartitioner() { - @Override - public Integer partition(final String topic, final String key, final Object value, final int numPartitions) { - return 1; - } - }; + metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster); + partitioner = (topic, key, value, numPartitions) -> 1; } @Test - public void shouldNotThrowNPEWhenOnChangeNotCalled() { + public void shouldNotThrowExceptionWhenOnChangeNotCalled() { new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne).getAllMetadataForStore("store"); } @Test public void shouldGetAllStreamInstances() { - final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), Collections.emptySet(), Collections.emptySet()); - final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), Collections.emptySet(), Collections.emptySet()); - final StreamsMetadata three = new StreamsMetadata(hostThree, Utils.mkSet(globalTable, "table-three"), - Collections.singleton(topic3P0), Collections.emptySet(), Collections.emptySet()); + final StreamsMetadata one = new StreamsMetadata(hostOne, + Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), + Utils.mkSet(topic1P0, topic2P1, topic4P0), + Utils.mkSet("table-one", "table-two", "merged-table"), + Utils.mkSet(topic2P0, topic1P1)); + final StreamsMetadata two = new StreamsMetadata(hostTwo, + Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), + Utils.mkSet(topic2P0, topic1P1), + Utils.mkSet("table-three"), + Utils.mkSet(topic3P0)); + final StreamsMetadata three = new StreamsMetadata(hostThree, + Utils.mkSet(globalTable, "table-three"), + Collections.singleton(topic3P0), + Utils.mkSet("table-one", "table-two", "merged-table"), + Utils.mkSet(topic1P0, topic2P1, topic4P0)); final Collection actual = metadataState.getAllMetadata(); assertEquals(3, actual.size()); @@ -155,18 +156,13 @@ public void shouldGetAllStreamInstances() { @Test public void shouldGetAllStreamsInstancesWithNoStores() { - builder.stream("topic-five").filter(new Predicate() { - @Override - public boolean test(final Object key, final Object value) { - return true; - } - }).to("some-other-topic"); + builder.stream("topic-five").filter((key, value) -> true).to("some-other-topic"); final TopicPartition tp5 = new TopicPartition("topic-five", 1); final HostInfo hostFour = new HostInfo("host-four", 8080); - hostToPartitions.put(hostFour, Utils.mkSet(tp5)); + hostToActivePartitions.put(hostFour, Utils.mkSet(tp5)); - metadataState.onChange(hostToPartitions, Collections.emptyMap(), + metadataState.onChange(hostToActivePartitions, Collections.emptyMap(), cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); final StreamsMetadata expected = new StreamsMetadata(hostFour, Collections.singleton(globalTable), @@ -177,14 +173,24 @@ public boolean test(final Object key, final Object value) { @Test public void shouldGetInstancesForStoreName() { - final StreamsMetadata one = new StreamsMetadata(hostOne, Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), Collections.emptySet(), Collections.emptySet()); - final StreamsMetadata two = new StreamsMetadata(hostTwo, Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), Collections.emptySet(), Collections.emptySet()); + final StreamsMetadata one = new StreamsMetadata(hostOne, + Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), + Utils.mkSet(topic1P0, topic2P1, topic4P0), + Utils.mkSet("table-one", "table-two", "merged-table"), + Utils.mkSet(topic2P0, topic1P1)); + final StreamsMetadata two = new StreamsMetadata(hostTwo, + Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), + Utils.mkSet(topic2P0, topic1P1), + Utils.mkSet("table-three"), + Utils.mkSet(topic3P0)); final Collection actual = metadataState.getAllMetadataForStore("table-one"); - assertEquals(2, actual.size()); + final Map actualAsMap = actual.stream() + .collect(Collectors.toMap(StreamsMetadata::hostInfo, Function.identity())); + assertEquals(3, actual.size()); assertTrue("expected " + actual + " to contain " + one, actual.contains(one)); assertTrue("expected " + actual + " to contain " + two, actual.contains(two)); + assertTrue("expected " + hostThree + " to contain as standby", + actualAsMap.get(hostThree).standbyStateStoreNames().contains("table-one")); } @Test(expected = NullPointerException.class) @@ -201,39 +207,38 @@ public void shouldReturnEmptyCollectionOnGetAllInstancesWithStoreWhenStoreDoesnt @Test public void shouldGetInstanceWithKey() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); - hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); + hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); - metadataState.onChange(hostToPartitions, Collections.emptyMap(), + metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); - final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, Collections.emptySet(), 0); - + final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, Utils.mkSet(hostTwo), 0); final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("table-three", "the-key", Serdes.String().serializer()); - assertEquals(expected, actual); } @Test public void shouldGetInstanceWithKeyAndCustomPartitioner() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); - hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); + hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); - metadataState.onChange(hostToPartitions, Collections.emptyMap(), + metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); - final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 1); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 1); final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("table-three", "the-key", partitioner); assertEquals(expected, actual); + assertEquals(1, actual.getPartition()); } @Test public void shouldReturnNotAvailableWhenClusterIsEmpty() { - metadataState.onChange(Collections.>emptyMap(), Collections.emptyMap(), Cluster.empty()); + metadataState.onChange(Collections.emptyMap(), Collections.emptyMap(), Cluster.empty()); final KeyQueryMetadata result = metadataState.getKeyQueryMetadataForKey("table-one", "a", Serdes.String().serializer()); assertEquals(KeyQueryMetadata.NOT_AVAILABLE, result); } @@ -241,21 +246,17 @@ public void shouldReturnNotAvailableWhenClusterIsEmpty() { @Test public void shouldGetInstanceWithKeyWithMergedStreams() { final TopicPartition topic2P2 = new TopicPartition("topic-two", 2); - hostToPartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1, topic2P2)); - metadataState.onChange(hostToPartitions, Collections.emptyMap(), + hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1, topic2P2)); + hostToStandbyPartitions.put(hostOne, Utils.mkSet(topic2P0, topic1P1, topic2P2)); + metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); - final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Collections.emptySet(), 2); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Utils.mkSet(hostOne), 2); - final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("merged-table", "the-key", new StreamPartitioner() { - @Override - public Integer partition(final String topic, final String key, final Object value, final int numPartitions) { - return 2; - } - }); + final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("merged-table", "the-key", + (topic, key, value, numPartitions) -> 2); assertEquals(expected, actual); - } @Test @@ -305,7 +306,7 @@ public void shouldGetMyMetadataForGlobalStoreWithKey() { @Test public void shouldGetAnyHostForGlobalStoreByKeyIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); - streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); + streamsMetadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster); assertNotNull(streamsMetadataState.getKeyQueryMetadataForKey(globalTable, "key", Serdes.String().serializer())); } @@ -318,9 +319,7 @@ public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { @Test public void shouldGetAnyHostForGlobalStoreByKeyAndPartitionerIfMyHostUnknown() { final StreamsMetadataState streamsMetadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), StreamsMetadataState.UNKNOWN_HOST); - streamsMetadataState.onChange(hostToPartitions, Collections.emptyMap(), cluster); + streamsMetadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster); assertNotNull(streamsMetadataState.getKeyQueryMetadataForKey(globalTable, "key", partitioner)); } - - } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java index 0050ff5e63d50..ab566d196be78 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignorTest.java @@ -17,6 +17,8 @@ package org.apache.kafka.streams.processor.internals; import java.util.Arrays; +import java.util.Collection; +import java.util.stream.Collectors; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; @@ -883,15 +885,20 @@ public void testAssignWithStandbyReplicas() { builder.addSource(null, "source2", null, null, null, "topic2"); builder.addProcessor("processor", new MockProcessorSupplier(), "source1", "source2"); final List topics = asList("topic1", "topic2"); + final Set allTopicPartitions = topics.stream() + .map(topic -> asList(new TopicPartition(topic, 0), new TopicPartition(topic, 1), new TopicPartition(topic, 2))) + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + final Set allTasks = mkSet(task0_0, task0_1, task0_2); final Set prevTasks00 = mkSet(task0_0); final Set prevTasks01 = mkSet(task0_1); final Set prevTasks02 = mkSet(task0_2); + final Set standbyTasks00 = mkSet(task0_0); final Set standbyTasks01 = mkSet(task0_1); final Set standbyTasks02 = mkSet(task0_2); - final Set standbyTasks00 = mkSet(task0_0); final UUID uuid1 = UUID.randomUUID(); final UUID uuid2 = UUID.randomUUID(); @@ -906,11 +913,11 @@ public void testAssignWithStandbyReplicas() { subscriptions.put("consumer10", new ConsumerPartitionAssignor.Subscription( topics, - getInfo(uuid1, prevTasks00, standbyTasks01).encode())); + getInfo(uuid1, prevTasks00, standbyTasks01, "any:9096").encode())); subscriptions.put("consumer11", new ConsumerPartitionAssignor.Subscription( topics, - getInfo(uuid1, prevTasks01, standbyTasks02).encode())); + getInfo(uuid1, prevTasks01, standbyTasks02, "any:9096").encode())); subscriptions.put("consumer20", new ConsumerPartitionAssignor.Subscription( topics, @@ -941,12 +948,32 @@ public void testAssignWithStandbyReplicas() { allStandbyTasks.addAll(info20.standbyTasks().keySet()); // all task ids are in the active tasks and also in the standby tasks - assertEquals(3, allActiveTasks.size()); assertEquals(allTasks, allActiveTasks); assertEquals(3, allStandbyTasks.size()); assertEquals(allTasks, allStandbyTasks); + + // Check host partition assignments + final Map> partitionsByHost = info10.partitionsByHost(); + assertEquals(2, partitionsByHost.size()); + assertEquals(allTopicPartitions, partitionsByHost.values().stream() + .flatMap(Collection::stream).collect(Collectors.toSet())); + + final Map> standbyPartitionsByHost = info10.standbyPartitionByHost(); + assertEquals(2, standbyPartitionsByHost.size()); + assertEquals(allTopicPartitions, standbyPartitionsByHost.values().stream() + .flatMap(Collection::stream).collect(Collectors.toSet())); + + for (final HostInfo hostInfo : partitionsByHost.keySet()) { + assertTrue(Collections.disjoint(partitionsByHost.get(hostInfo), standbyPartitionsByHost.get(hostInfo))); + } + + // All consumers got the same host info + assertEquals(partitionsByHost, info11.partitionsByHost()); + assertEquals(partitionsByHost, info20.partitionsByHost()); + assertEquals(standbyPartitionsByHost, info11.standbyPartitionByHost()); + assertEquals(standbyPartitionsByHost, info20.standbyPartitionByHost()); } @Test @@ -1331,12 +1358,17 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() { final Map assign = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); final ConsumerPartitionAssignor.Assignment consumer1Assignment = assign.get("consumer1"); final AssignmentInfo assignmentInfo = AssignmentInfo.decode(consumer1Assignment.userData()); - final Set consumer1partitions = assignmentInfo.partitionsByHost().get(new HostInfo("localhost", 8080)); - final Set consumer2Partitions = assignmentInfo.partitionsByHost().get(new HostInfo("other", 9090)); - final HashSet allAssignedPartitions = new HashSet<>(consumer1partitions); - allAssignedPartitions.addAll(consumer2Partitions); - assertThat(consumer1partitions, not(allPartitions)); - assertThat(consumer2Partitions, not(allPartitions)); + + final Set consumer1ActivePartitions = assignmentInfo.partitionsByHost().get(new HostInfo("localhost", 8080)); + final Set consumer2ActivePartitions = assignmentInfo.partitionsByHost().get(new HostInfo("other", 9090)); + final Set consumer1StandbyPartitions = assignmentInfo.standbyPartitionByHost().get(new HostInfo("localhost", 8080)); + final Set consumer2StandbyPartitions = assignmentInfo.standbyPartitionByHost().get(new HostInfo("other", 9090)); + final HashSet allAssignedPartitions = new HashSet<>(consumer1ActivePartitions); + allAssignedPartitions.addAll(consumer2ActivePartitions); + assertThat(consumer1ActivePartitions, not(allPartitions)); + assertThat(consumer2ActivePartitions, not(allPartitions)); + assertThat(consumer1ActivePartitions, equalTo(consumer2StandbyPartitions)); + assertThat(consumer2ActivePartitions, equalTo(consumer1StandbyPartitions)); assertThat(allAssignedPartitions, equalTo(allPartitions)); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index bc9a538d40251..c7ed534af79c8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -36,78 +36,99 @@ public class AssignmentInfoTest { private final List activeTasks = Arrays.asList( new TaskId(0, 0), - new TaskId(0, 0), - new TaskId(0, 1), new TaskId(1, 0)); + new TaskId(0, 1), + new TaskId(1, 0), + new TaskId(1, 1)); private final Map> standbyTasks = new HashMap>() { { + put(new TaskId(1, 0), + Utils.mkSet(new TopicPartition("t1", 0), new TopicPartition("t2", 0))); put(new TaskId(1, 1), Utils.mkSet(new TopicPartition("t1", 1), new TopicPartition("t2", 1))); - put(new TaskId(2, 0), - Utils.mkSet(new TopicPartition("t3", 0), new TopicPartition("t3", 0))); } }; - private final Map> globalAssignment = new HashMap>() { + private final Map> activeAssignment = new HashMap>() { + { + put(new HostInfo("localhost", 8088), Utils.mkSet( + new TopicPartition("t0", 0), + new TopicPartition("t1", 0), + new TopicPartition("t2", 0))); + put(new HostInfo("localhost", 8089), Utils.mkSet( + new TopicPartition("t0", 1), + new TopicPartition("t1", 1), + new TopicPartition("t2", 1))); + } + }; + private final Map> standbyAssignment = new HashMap>() { { - put(new HostInfo("localhost", 80), - Utils.mkSet(new TopicPartition("t1", 1), new TopicPartition("t3", 3))); + put(new HostInfo("localhost", 8088), Utils.mkSet( + new TopicPartition("t1", 0), + new TopicPartition("t2", 0))); + put(new HostInfo("localhost", 8089), Utils.mkSet( + new TopicPartition("t1", 1), + new TopicPartition("t2", 1))); } }; @Test public void shouldUseLatestSupportedVersionByDefault() { - final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 0); assertEquals(LATEST_SUPPORTED_VERSION, info.version()); } @Test(expected = IllegalArgumentException.class) public void shouldThrowForUnknownVersion1() { - new AssignmentInfo(0, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + new AssignmentInfo(0, activeTasks, standbyTasks, activeAssignment, Collections.emptyMap(), 0); } @Test(expected = IllegalArgumentException.class) public void shouldThrowForUnknownVersion2() { - new AssignmentInfo(LATEST_SUPPORTED_VERSION + 1, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + new AssignmentInfo(LATEST_SUPPORTED_VERSION + 1, activeTasks, standbyTasks, activeAssignment, Collections.emptyMap(), 0); } @Test public void shouldEncodeAndDecodeVersion1() { - final AssignmentInfo info = new AssignmentInfo(1, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo info = new AssignmentInfo(1, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 0); final AssignmentInfo expectedInfo = new AssignmentInfo(1, UNKNOWN, activeTasks, standbyTasks, Collections.emptyMap(), Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion2() { - final AssignmentInfo info = new AssignmentInfo(2, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); - final AssignmentInfo expectedInfo = new AssignmentInfo(2, UNKNOWN, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo info = new AssignmentInfo(2, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 0); + final AssignmentInfo expectedInfo = new AssignmentInfo(2, UNKNOWN, activeTasks, standbyTasks, activeAssignment, Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion3() { - final AssignmentInfo info = new AssignmentInfo(3, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); - final AssignmentInfo expectedInfo = new AssignmentInfo(3, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 0); + final AssignmentInfo info = new AssignmentInfo(3, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 0); + final AssignmentInfo expectedInfo = new AssignmentInfo(3, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, + activeAssignment, Collections.emptyMap(), 0); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion4() { - final AssignmentInfo info = new AssignmentInfo(4, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(4, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo info = new AssignmentInfo(4, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(4, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, + activeAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion5() { - final AssignmentInfo info = new AssignmentInfo(5, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(5, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo info = new AssignmentInfo(5, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(5, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, + activeAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion6() { - final AssignmentInfo info = new AssignmentInfo(6, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(6, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo info = new AssignmentInfo(6, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(6, LATEST_SUPPORTED_VERSION, activeTasks, standbyTasks, + activeAssignment, standbyAssignment, 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } @@ -115,8 +136,10 @@ public void shouldEncodeAndDecodeVersion6() { public void shouldEncodeAndDecodeSmallerCommonlySupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; final int commonlySupportedVersion = LATEST_SUPPORTED_VERSION - 1; - final AssignmentInfo info = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); - final AssignmentInfo expectedInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, globalAssignment, Collections.emptyMap(), 2); + final AssignmentInfo info = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, + activeAssignment, standbyAssignment, 2); + final AssignmentInfo expectedInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, + activeAssignment, Collections.emptyMap(), 2); assertEquals(expectedInfo, AssignmentInfo.decode(info.encode())); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java index 159db4eb1947a..a1adb638c203f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/tests/StreamsUpgradeTest.java @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.tests; -import java.util.Collections; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; @@ -226,7 +225,7 @@ public void onAssignment(final ConsumerPartitionAssignor.Assignment assignment, final TaskManager taskManager = taskManger(); taskManager.setClusterMetadata(Cluster.empty().withPartitions(topicToPartitionInfo)); - taskManager.setHostPartitionMappings(partitionsByHost, Collections.emptyMap()); + taskManager.setHostPartitionMappings(partitionsByHost, info.standbyPartitionByHost()); taskManager.setPartitionsToTaskId(partitionsToTaskId); taskManager.setAssignmentMetadata(activeTasks, info.standbyTasks()); taskManager.updateSubscriptionsFromAssignment(partitions); From 5cebfa9c8efbe68b7cac99937108b847e1580dc3 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Wed, 25 Dec 2019 14:09:01 +0530 Subject: [PATCH 12/21] Fixing shouldApplyUpdatesToStandbyStore test case --- .../integration/OptimizedKTableIntegrationTest.java | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java index a53f57dd63aed..e2ef3f13f1531 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/OptimizedKTableIntegrationTest.java @@ -49,6 +49,7 @@ import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.Consumed; import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.processor.StateRestoreListener; import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.QueryableStoreTypes; @@ -64,7 +65,7 @@ @Category(IntegrationTest.class) public class OptimizedKTableIntegrationTest { private static final int NUM_BROKERS = 1; - + private static int port = 0; private static final String INPUT_TOPIC_NAME = "input-topic"; private static final String TABLE_NAME = "source-table"; @@ -157,7 +158,9 @@ public void shouldApplyUpdatesToStandbyStore() throws Exception { .store(TABLE_NAME, QueryableStoreTypes.keyValueStore()); final boolean kafkaStreams1WasFirstActive; - if (store1.get(key) != null) { + final KeyQueryMetadata keyQueryMetadata = kafkaStreams1.queryMetadataForKey(TABLE_NAME, key, (topic, somekey, value, numPartitions) -> 0); + + if ((keyQueryMetadata.getActiveHost().port() % 2) == 1) { kafkaStreams1WasFirstActive = true; } else { // Assert that data from the job was sent to the store @@ -254,6 +257,7 @@ private Properties streamsConfiguration() { final Properties config = new Properties(); config.put(StreamsConfig.TOPOLOGY_OPTIMIZATION, StreamsConfig.OPTIMIZE); config.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); + config.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + String.valueOf(++port)); config.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); config.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory(applicationId).getPath()); config.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass()); From b8e9ea3d3e72df138fdf3a5ee1d689c411b434c3 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sun, 29 Dec 2019 01:50:24 +0530 Subject: [PATCH 13/21] Addressing code review comments --- .../apache/kafka/streams/KafkaStreams.java | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) 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 7099e73ced35a..70f2024d31c7b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -136,6 +136,7 @@ public class KafkaStreams implements AutoCloseable { private static final String JMX_PREFIX = "kafka.streams"; + private static final long UNKNOWN_POSITION = -1; // processId is expected to be unique across JVMs and to be used // in userData of the subscription request to allow assignor be aware @@ -1167,7 +1168,6 @@ public Map> allLocalOffsetLags() { final Map> localOffsetLags = new HashMap<>(); final Map standbyChangelogPositions = new HashMap<>(); final Map activeChangelogPositions = new HashMap<>(); - final long unknownPosition = 0; // Obtain the current positions, of all the active-restoring and standby tasks for (final StreamThread streamThread : this.threads) { @@ -1175,7 +1175,7 @@ public Map> allLocalOffsetLags() { final Map changelogPartitionLimits = standbyTask.checkpointedOffsets(); standbyTask.changelogPartitions().forEach(topicPartition -> standbyChangelogPositions.put(topicPartition, - changelogPartitionLimits.getOrDefault(topicPartition, unknownPosition))); + changelogPartitionLimits.getOrDefault(topicPartition, UNKNOWN_POSITION))); } final Set restoringTaskIds = streamThread.restoringTaskIds(); @@ -1184,9 +1184,9 @@ public Map> allLocalOffsetLags() { final Map restoredOffsets = activeTask.restoredOffsets(); activeTask.changelogPartitions().forEach(topicPartition -> { if (isRestoring) { - activeChangelogPositions.put(topicPartition, restoredOffsets.getOrDefault(topicPartition, unknownPosition)); + activeChangelogPositions.put(topicPartition, restoredOffsets.getOrDefault(topicPartition, UNKNOWN_POSITION)); } else { - activeChangelogPositions.put(topicPartition, unknownPosition); + activeChangelogPositions.put(topicPartition, UNKNOWN_POSITION); } }); } @@ -1196,19 +1196,23 @@ public Map> allLocalOffsetLags() { final Map offsetSpecMap = new HashMap<>(); Stream.concat(activeChangelogPositions.keySet().stream(), standbyChangelogPositions.keySet().stream()) .forEach(topicPartition -> offsetSpecMap.put(topicPartition, OffsetSpec.latest())); - try { - final Map allEndOffsets = adminClient.listOffsets(offsetSpecMap).all().get(); + final Map allEndOffsets = new HashMap<>(); + try { + allEndOffsets.putAll(adminClient.listOffsets(offsetSpecMap).all().get()); + } catch (final Exception e) { + throw new StreamsException("Unable to obtain end offsets from kafka", e); + } log.info("Current end offsets :" + allEndOffsets); allEndOffsets.forEach((topicPartition, offsetsResultInfo) -> { final String storeName = streamsMetadataState.getStoreForChangelogTopic(topicPartition.topic()); final long offsetPosition; if (activeChangelogPositions.containsKey(topicPartition)) { // if unknown, assume it's positioned at the tail of changelog partition - offsetPosition = activeChangelogPositions.get(topicPartition) == unknownPosition ? + offsetPosition = activeChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? offsetsResultInfo.offset() : activeChangelogPositions.get(topicPartition); } else if (standbyChangelogPositions.containsKey(topicPartition)) { // if unknown, assume it's positioned at the head of changelog partition - offsetPosition = standbyChangelogPositions.get(topicPartition) == unknownPosition ? + offsetPosition = standbyChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? 0 : standbyChangelogPositions.get(topicPartition); } else { throw new IllegalStateException("Topic Partition " + topicPartition + " should be either active or standby"); @@ -1225,9 +1229,6 @@ public Map> allLocalOffsetLags() { } localOffsetLags.put(storeName, partitionToOffsetLag); }); - } catch (final Exception e) { - throw new StreamsException("Unable to obtain end offsets from kafka", e); - } return localOffsetLags; } From d776cf1ec70d43fc69d81f6f0c01380089f38e94 Mon Sep 17 00:00:00 2001 From: Navinder Brar Date: Sun, 29 Dec 2019 02:31:56 +0530 Subject: [PATCH 14/21] Fixing the checkstylese --- .../apache/kafka/streams/KafkaStreams.java | 56 +++++++++---------- 1 file changed, 28 insertions(+), 28 deletions(-) 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 70f2024d31c7b..762cce3d76f92 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1196,39 +1196,39 @@ public Map> allLocalOffsetLags() { final Map offsetSpecMap = new HashMap<>(); Stream.concat(activeChangelogPositions.keySet().stream(), standbyChangelogPositions.keySet().stream()) .forEach(topicPartition -> offsetSpecMap.put(topicPartition, OffsetSpec.latest())); - final Map allEndOffsets = new HashMap<>(); - try { - allEndOffsets.putAll(adminClient.listOffsets(offsetSpecMap).all().get()); - } catch (final Exception e) { - throw new StreamsException("Unable to obtain end offsets from kafka", e); - } - log.info("Current end offsets :" + allEndOffsets); - allEndOffsets.forEach((topicPartition, offsetsResultInfo) -> { - final String storeName = streamsMetadataState.getStoreForChangelogTopic(topicPartition.topic()); - final long offsetPosition; - if (activeChangelogPositions.containsKey(topicPartition)) { - // if unknown, assume it's positioned at the tail of changelog partition - offsetPosition = activeChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? + final Map allEndOffsets = new HashMap<>(); + try { + allEndOffsets.putAll(adminClient.listOffsets(offsetSpecMap).all().get()); + } catch (final Exception e) { + throw new StreamsException("Unable to obtain end offsets from kafka", e); + } + log.info("Current end offsets :" + allEndOffsets); + allEndOffsets.forEach((topicPartition, offsetsResultInfo) -> { + final String storeName = streamsMetadataState.getStoreForChangelogTopic(topicPartition.topic()); + final long offsetPosition; + if (activeChangelogPositions.containsKey(topicPartition)) { + // if unknown, assume it's positioned at the tail of changelog partition + offsetPosition = activeChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? offsetsResultInfo.offset() : activeChangelogPositions.get(topicPartition); - } else if (standbyChangelogPositions.containsKey(topicPartition)) { - // if unknown, assume it's positioned at the head of changelog partition - offsetPosition = standbyChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? + } else if (standbyChangelogPositions.containsKey(topicPartition)) { + // if unknown, assume it's positioned at the head of changelog partition + offsetPosition = standbyChangelogPositions.get(topicPartition) == UNKNOWN_POSITION ? 0 : standbyChangelogPositions.get(topicPartition); - } else { - throw new IllegalStateException("Topic Partition " + topicPartition + " should be either active or standby"); - } + } else { + throw new IllegalStateException("Topic Partition " + topicPartition + " should be either active or standby"); + } - final long offsetLag = offsetsResultInfo.offset() - offsetPosition; - final Map partitionToOffsetLag = localOffsetLags + final long offsetLag = offsetsResultInfo.offset() - offsetPosition; + final Map partitionToOffsetLag = localOffsetLags .getOrDefault(storeName, new HashMap<>()); - if (!partitionToOffsetLag.containsKey(topicPartition.partition())) { - partitionToOffsetLag.put(topicPartition.partition(), offsetLag); - } else { - throw new IllegalStateException("Encountered the same store partition" + storeName + "," + if (!partitionToOffsetLag.containsKey(topicPartition.partition())) { + partitionToOffsetLag.put(topicPartition.partition(), offsetLag); + } else { + throw new IllegalStateException("Encountered the same store partition" + storeName + "," + topicPartition.partition() + " more than once"); - } - localOffsetLags.put(storeName, partitionToOffsetLag); - }); + } + localOffsetLags.put(storeName, partitionToOffsetLag); + }); return localOffsetLags; } From fe7b70d0c8d5c826af50935d21132021b1ceb9aa Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 30 Dec 2019 11:30:53 -0800 Subject: [PATCH 15/21] Changing few other maps to concurrent maps for safer iteration --- .../streams/processor/internals/ProcessorStateManager.java | 5 +++-- .../streams/processor/internals/StoreChangelogReader.java | 3 ++- 2 files changed, 5 insertions(+), 3 deletions(-) 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 87ee9607ad603..6aa0a42335501 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 @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.concurrent.ConcurrentHashMap; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.utils.FixedOrderMap; @@ -73,7 +74,7 @@ public class ProcessorStateManager implements StateManager { private final boolean eosEnabled; private final File baseDir; private OffsetCheckpoint checkpointFile; - private final Map checkpointFileCache = new HashMap<>(); + private final Map checkpointFileCache = new ConcurrentHashMap<>(); private final Map initialLoadedCheckpoints; /** @@ -102,7 +103,7 @@ public ProcessorStateManager(final TaskId taskId, offsetLimits = new HashMap<>(); standbyRestoredOffsets = new HashMap<>(); this.isStandby = isStandby; - restoreCallbacks = isStandby ? new HashMap<>() : null; + restoreCallbacks = isStandby ? new ConcurrentHashMap<>() : null; recordConverters = isStandby ? new HashMap<>() : null; this.storeToChangelogTopic = new HashMap<>(storeToChangelogTopic); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java index 669eabf8bb5fc..b8be4bc081e8f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StoreChangelogReader.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import java.util.concurrent.ConcurrentHashMap; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; @@ -46,7 +47,7 @@ public class StoreChangelogReader implements ChangelogReader { private final StateRestoreListener userStateRestoreListener; private final Map restoreToOffsets = new HashMap<>(); private final Map> partitionInfo = new HashMap<>(); - private final Map stateRestorers = new HashMap<>(); + private final Map stateRestorers = new ConcurrentHashMap<>(); private final Set needsRestoring = new HashSet<>(); private final Set needsInitializing = new HashSet<>(); private final Set completedRestorers = new HashSet<>(); From e62c06463a8317e7851aa9280cb926b58106f987 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 6 Jan 2020 20:25:45 -0800 Subject: [PATCH 16/21] Adding tests for KafkaStreams#allLocalOffsetLags() --- .../kafka/clients/admin/MockAdminClient.java | 9 ++++-- .../apache/kafka/streams/KafkaStreams.java | 2 +- .../kafka/streams/KafkaStreamsTest.java | 30 +++++++++++++++++++ .../QueryableStateIntegrationTest.java | 28 ++++++++++++++++- 4 files changed, 65 insertions(+), 4 deletions(-) diff --git a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java index 7cfd0510cc767..284d4f68308c0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java +++ b/clients/src/test/java/org/apache/kafka/clients/admin/MockAdminClient.java @@ -450,12 +450,17 @@ public ListPartitionReassignmentsResult listPartitionReassignments(Optional offsets, AlterConsumerGroupOffsetsOptions options) { - throw new UnsupportedOperationException("Not implement yet"); + throw new UnsupportedOperationException("Not implemented yet"); } @Override public ListOffsetsResult listOffsets(Map topicPartitionOffsets, ListOffsetsOptions options) { - throw new UnsupportedOperationException("Not implement yet"); + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public ListOffsetsResult listOffsets(Map topicPartitionOffsets) { + throw new UnsupportedOperationException("Not implemented yet"); } @Override 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 762cce3d76f92..42ac7c4f8b11e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1230,7 +1230,7 @@ public Map> allLocalOffsetLags() { localOffsetLags.put(storeName, partitionToOffsetLag); }); - return localOffsetLags; + return Collections.unmodifiableMap(localOffsetLags); } /** 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 4cd427a78eeb2..6af76e9d4325e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -17,9 +17,14 @@ package org.apache.kafka.streams; import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ListOffsetsResult; +import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; +import org.apache.kafka.clients.admin.MockAdminClient; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.common.Cluster; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.internals.KafkaFutureImpl; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.metrics.MetricsReporter; @@ -313,6 +318,10 @@ private void prepareStreamThread(final StreamThread thread, final boolean termin Thread.sleep(50L); return null; }).anyTimes(); + + EasyMock.expect(thread.allStandbyTasks()).andStubReturn(Collections.emptyList()); + EasyMock.expect(thread.restoringTaskIds()).andStubReturn(Collections.emptySet()); + EasyMock.expect(thread.allStreamsTasks()).andStubReturn(Collections.emptyList()); } @Test @@ -677,6 +686,27 @@ public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing( streams.queryMetadataForKey("store", "key", (topic, key, value, numPartitions) -> 0); } + @Test + public void shouldReturnEmptyLocalOffsetLags() { + // Mock all calls made to compute the offset lags, + final ListOffsetsResult result = EasyMock.mock(ListOffsetsResult.class); + final KafkaFutureImpl> allFuture = new KafkaFutureImpl<>(); + allFuture.complete(Collections.emptyMap()); + + EasyMock.expect(result.all()).andReturn(allFuture); + final MockAdminClient mockAdminClient = EasyMock.partialMockBuilder(MockAdminClient.class) + .addMockedMethod("listOffsets", Map.class).createMock(); + EasyMock.expect(mockAdminClient.listOffsets(anyObject())).andStubReturn(result); + final MockClientSupplier mockClientSupplier = EasyMock.partialMockBuilder(MockClientSupplier.class) + .addMockedMethod("getAdmin").createMock(); + EasyMock.expect(mockClientSupplier.getAdmin(anyObject())).andReturn(mockAdminClient); + EasyMock.replay(result, mockAdminClient, mockClientSupplier); + + final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, mockClientSupplier, time); + streams.start(); + assertEquals(0, streams.allLocalOffsetLags().size()); + } + @Test public void shouldReturnFalseOnCloseWhenThreadsHaventTerminated() { // do not use mock time so that it can really elapse 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 2148fe829e62f..98d91cc22cc07 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 @@ -19,6 +19,7 @@ import static java.time.Duration.ofMillis; import static java.time.Duration.ofSeconds; import static java.time.Instant.ofEpochMilli; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.waitForApplicationState; import static org.apache.kafka.test.StreamsTestUtils.startKafkaStreamsAndWaitForRunningState; @@ -260,6 +261,19 @@ private KafkaStreams createCountStream(final String inputTopic, return new KafkaStreams(builder.build(), streamsConfiguration); } + private void verifyOffsetLagFetch(final List streamsList, + final Set stores, + final List partitionsPerStreamsInstance) { + for (int i = 0; i < streamsList.size(); i++) { + final Map> localOffsetLags = streamsList.get(i).allLocalOffsetLags(); + final int expectedPartitions = partitionsPerStreamsInstance.get(i); + assertThat(localOffsetLags.values().stream().mapToInt(Map::size).sum(), equalTo(expectedPartitions)); + if (expectedPartitions > 0) { + assertThat(localOffsetLags.keySet(), equalTo(stores)); + } + } + } + private void verifyAllKVKeys(final List streamsList, final KafkaStreams streams, final KafkaStreamsTest.StateListenerStub stateListener, @@ -404,7 +418,7 @@ private void assertNoKVKeyFailures(final String storeName, } @Test - public void queryOnRebalance() throws Exception { + public void shouldBeAbleToQueryDuringRebalance() throws Exception { final int numThreads = STREAM_TWO_PARTITIONS; final List streamsList = new ArrayList<>(numThreads); final List listeners = new ArrayList<>(numThreads); @@ -428,6 +442,9 @@ public void queryOnRebalance() throws Exception { } startApplicationAndWaitUntilRunning(streamsList, Duration.ofSeconds(60)); + final Set stores = mkSet(storeName + "-" + streamThree, windowStoreName + "-" + streamThree); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(4, 4)); + try { waitUntilAtLeastNumRecordProcessed(outputTopicThree, 1); @@ -451,6 +468,7 @@ public void queryOnRebalance() throws Exception { DEFAULT_TIMEOUT_MS, true); } + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(4, 4)); // kill N-1 threads for (int i = 1; i < streamsList.size(); i++) { @@ -460,10 +478,12 @@ public void queryOnRebalance() throws Exception { } waitForApplicationState(streamsList.subList(1, numThreads), State.NOT_RUNNING, Duration.ofSeconds(60)); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(4, 0)); // It's not enough to assert that the first instance is RUNNING because it is possible // for the above checks to succeed while the instance is in a REBALANCING state. waitForApplicationState(streamsList.subList(0, 1), State.RUNNING, Duration.ofSeconds(60)); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(4, 0)); // Even though the closed instance(s) are now in NOT_RUNNING there is no guarantee that // the running instance is aware of this, so we must run our follow up queries with @@ -488,6 +508,7 @@ public void queryOnRebalance() throws Exception { WINDOW_SIZE, DEFAULT_TIMEOUT_MS, true); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(8, 0)); } finally { for (final KafkaStreams streams : streamsList) { streams.close(); @@ -522,6 +543,8 @@ public void shouldBeAbleQueryStandbyStateDuringRebalance() throws Exception { streamsList.add(streams); } startApplicationAndWaitUntilRunning(streamsList, Duration.ofSeconds(60)); + final Set stores = mkSet(storeName + "-" + streamThree, windowStoreName + "-" + streamThree); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(8, 8)); try { waitUntilAtLeastNumRecordProcessed(outputTopicThree, 1); @@ -547,6 +570,7 @@ public void shouldBeAbleQueryStandbyStateDuringRebalance() throws Exception { DEFAULT_TIMEOUT_MS, false); } + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(8, 8)); // kill N-1 threads for (int i = 1; i < streamsList.size(); i++) { @@ -556,6 +580,7 @@ public void shouldBeAbleQueryStandbyStateDuringRebalance() throws Exception { } waitForApplicationState(streamsList.subList(1, numThreads), State.NOT_RUNNING, Duration.ofSeconds(60)); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(8, 0)); // Now, confirm that all the keys are still queryable on the remaining thread, regardless of the state verifyAllKVKeys( @@ -576,6 +601,7 @@ public void shouldBeAbleQueryStandbyStateDuringRebalance() throws Exception { WINDOW_SIZE, DEFAULT_TIMEOUT_MS, false); + verifyOffsetLagFetch(streamsList, stores, Arrays.asList(8, 0)); } finally { for (final KafkaStreams streams : streamsList) { streams.close(); From ed11a04ad363fa8121afc3ab2843ce338b094ef4 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Tue, 7 Jan 2020 14:43:29 -0800 Subject: [PATCH 17/21] Introduce a LagInfo class that holds both raw offset position, as well as offset lag - Improved javadocs for new class and KeyQueryMetadata - Redid the tests, based on new APIs --- .../apache/kafka/streams/KafkaStreams.java | 19 ++-- .../kafka/streams/KeyQueryMetadata.java | 22 ++++- .../org/apache/kafka/streams/LagInfo.java | 91 +++++++++++++++++++ .../kafka/streams/KafkaStreamsTest.java | 4 +- .../QueryableStateIntegrationTest.java | 7 +- 5 files changed, 124 insertions(+), 19 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/LagInfo.java 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 42ac7c4f8b11e..82b9275589725 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -1158,14 +1158,14 @@ public KeyQueryMetadata queryMetadataForKey(final String storeName, } /** - * Returns offset lag info, for all store partitions (active or standby) local to this Streams instance. Note that the + * Returns {@link LagInfo}, for all store partitions (active or standby) local to this Streams instance. Note that the * values returned are just estimates and meant to be used for making soft decisions on whether the data in the store * partition is fresh enough for querying. * - * @return map of store names to another map of partition to offset lags + * @return map of store names to another map of partition to {@link LagInfo}s */ - public Map> allLocalOffsetLags() { - final Map> localOffsetLags = new HashMap<>(); + public Map> allLocalStorePartitionLags() { + final Map> localStorePartitionLags = new HashMap<>(); final Map standbyChangelogPositions = new HashMap<>(); final Map activeChangelogPositions = new HashMap<>(); @@ -1218,19 +1218,18 @@ public Map> allLocalOffsetLags() { throw new IllegalStateException("Topic Partition " + topicPartition + " should be either active or standby"); } - final long offsetLag = offsetsResultInfo.offset() - offsetPosition; - final Map partitionToOffsetLag = localOffsetLags - .getOrDefault(storeName, new HashMap<>()); + final LagInfo lagInfo = new LagInfo(offsetPosition, offsetsResultInfo.offset()); + final Map partitionToOffsetLag = localStorePartitionLags.getOrDefault(storeName, new HashMap<>()); if (!partitionToOffsetLag.containsKey(topicPartition.partition())) { - partitionToOffsetLag.put(topicPartition.partition(), offsetLag); + partitionToOffsetLag.put(topicPartition.partition(), lagInfo); } else { throw new IllegalStateException("Encountered the same store partition" + storeName + "," + topicPartition.partition() + " more than once"); } - localOffsetLags.put(storeName, partitionToOffsetLag); + localStorePartitionLags.put(storeName, partitionToOffsetLag); }); - return Collections.unmodifiableMap(localOffsetLags); + return Collections.unmodifiableMap(localStorePartitionLags); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java index 8f723e43afed3..87520ed8a7e8f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java @@ -26,7 +26,7 @@ * Represents all the metadata related to a key, where a particular key resides in a {@link KafkaStreams} application. * It contains the active {@link HostInfo} and a set of standby {@link HostInfo}s, denoting the instances where the key resides. * It also contains the partition number where the key belongs, which could be useful when used in conjunction with other APIs. - * e.g: Relating with lags for that store partition using {@link KafkaStreams#allLocalOffsetLags()} + * e.g: Relating with lags for that store partition using {@link KafkaStreams#allLocalStorePartitionLags()} * NOTE: This is a point in time view. It may change as rebalances happen. */ public class KeyQueryMetadata { @@ -38,11 +38,10 @@ public class KeyQueryMetadata { Collections.emptySet(), -1); - // Active streams instance for key private final HostInfo activeHost; - // Streams instances that host the key as standbys + private final Set standbyHosts; - // Store partition corresponding to the key. + private final int partition; public KeyQueryMetadata(final HostInfo activeHost, final Set standbyHosts, final int partition) { @@ -51,14 +50,29 @@ public KeyQueryMetadata(final HostInfo activeHost, final Set standbyHo this.partition = partition; } + /** + * Get the Active streams instance for given key + * + * @return active instance's {@link HostInfo} + */ public HostInfo getActiveHost() { return activeHost; } + /** + * Get the Streams instances that host the key as standbys + * + * @return set of standby {@link HostInfo} or a empty set, if no standbys are configured + */ public Set getStandbyHosts() { return standbyHosts; } + /** + * Get the Store partition corresponding to the key. + * + * @return store partition number + */ public int getPartition() { return partition; } diff --git a/streams/src/main/java/org/apache/kafka/streams/LagInfo.java b/streams/src/main/java/org/apache/kafka/streams/LagInfo.java new file mode 100644 index 0000000000000..7148840b16e3f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/LagInfo.java @@ -0,0 +1,91 @@ +/* + * 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; + +import java.util.Objects; + +/** + * Encapsulates information about lag, at a store partition replica (active or standby). This information is constantly changing as the + * tasks process records and thus, they should be treated as simply instantaenous measure of lag. + */ +public class LagInfo { + + private final long currentOffsetPosition; + + private final long endOffsetPosition; + + private final long offsetLag; + + LagInfo(final long currentOffsetPosition, final long endOffsetPosition) { + this.currentOffsetPosition = currentOffsetPosition; + this.endOffsetPosition = endOffsetPosition; + this.offsetLag = Math.max(0, endOffsetPosition - currentOffsetPosition); + } + + /** + * Get the current maximum offset on the store partition's changelog topic, that has been successfully written into + * the store partition's state store. + * + * @return current consume offset for standby/restoring store partitions & simply endoffset for active store partition replicas + */ + public long currentOffsetPosition() { + return this.currentOffsetPosition; + } + + /** + * Get the end offset position for this store partition's changelog topic on the Kafka brokers. + * + * @return last offset written to the changelog topic partition + */ + public long endOffsetPosition() { + return this.endOffsetPosition; + } + + /** + * Get the measured lag between current and end offset positions, for this store partition replica + * + * @return lag as measured by message offsets + */ + public long offsetLag() { + return this.offsetLag; + } + + @Override + public boolean equals(final Object obj) { + if (!(obj instanceof LagInfo)) { + return false; + } + final LagInfo other = (LagInfo) obj; + return currentOffsetPosition == other.currentOffsetPosition + && endOffsetPosition == other.endOffsetPosition + && this.offsetLag == other.offsetLag; + } + + @Override + public int hashCode() { + return Objects.hash(currentOffsetPosition, endOffsetPosition, offsetLag); + } + + @Override + public String toString() { + return "LagInfo {" + + " currentOffsetPosition=" + currentOffsetPosition + + ", endOffsetPosition=" + endOffsetPosition + + ", offsetLag=" + offsetLag + + '}'; + } +} 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 6af76e9d4325e..d34a09d921bfc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KafkaStreamsTest.java @@ -687,7 +687,7 @@ public void shouldNotGetQueryMetadataWithPartitionerWhenNotRunningOrRebalancing( } @Test - public void shouldReturnEmptyLocalOffsetLags() { + public void shouldReturnEmptyLocalStorePartitionLags() { // Mock all calls made to compute the offset lags, final ListOffsetsResult result = EasyMock.mock(ListOffsetsResult.class); final KafkaFutureImpl> allFuture = new KafkaFutureImpl<>(); @@ -704,7 +704,7 @@ public void shouldReturnEmptyLocalOffsetLags() { final KafkaStreams streams = new KafkaStreams(new StreamsBuilder().build(), props, mockClientSupplier, time); streams.start(); - assertEquals(0, streams.allLocalOffsetLags().size()); + assertEquals(0, streams.allLocalStorePartitionLags().size()); } @Test 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 98d91cc22cc07..8a7d7c4c16f1a 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 @@ -73,6 +73,7 @@ import org.apache.kafka.streams.KafkaStreamsTest; import org.apache.kafka.streams.KeyQueryMetadata; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.LagInfo; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.errors.InvalidStateStoreException; @@ -265,11 +266,11 @@ private void verifyOffsetLagFetch(final List streamsList, final Set stores, final List partitionsPerStreamsInstance) { for (int i = 0; i < streamsList.size(); i++) { - final Map> localOffsetLags = streamsList.get(i).allLocalOffsetLags(); + final Map> localLags = streamsList.get(i).allLocalStorePartitionLags(); final int expectedPartitions = partitionsPerStreamsInstance.get(i); - assertThat(localOffsetLags.values().stream().mapToInt(Map::size).sum(), equalTo(expectedPartitions)); + assertThat(localLags.values().stream().mapToInt(Map::size).sum(), equalTo(expectedPartitions)); if (expectedPartitions > 0) { - assertThat(localOffsetLags.keySet(), equalTo(stores)); + assertThat(localLags.keySet(), equalTo(stores)); } } } From 1bc5a6492f98949fcf28ad2bc9ae7a7a5adbc9ed Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Wed, 8 Jan 2020 14:11:04 -0800 Subject: [PATCH 18/21] Cleanup code bases on CR comments --- .../apache/kafka/streams/KafkaStreams.java | 15 ++-- .../kafka/streams/KeyQueryMetadata.java | 2 +- .../apache/kafka/streams/StreamsConfig.java | 2 +- .../internals/StreamsMetadataState.java | 33 ++++---- .../internals/assignment/AssignmentInfo.java | 19 ++--- .../kafka/streams/state/StreamsMetadata.java | 79 +++++++++++-------- .../StreamThreadStateStoreProvider.java | 2 +- .../internals/StreamsMetadataStateTest.java | 4 +- .../assignment/AssignmentInfoTest.java | 55 ++++++------- 9 files changed, 113 insertions(+), 98 deletions(-) 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 82b9275589725..bdf87566c8e7f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams; +import java.util.function.Function; +import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.kafka.clients.admin.Admin; import org.apache.kafka.clients.admin.ListOffsetsResult.ListOffsetsResultInfo; @@ -1083,6 +1085,7 @@ public Collection allMetadataForStore(final String storeName) { * @param key type * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@code storeName} and * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing + * @deprecated Use {@link #queryMetadataForKey(String, Object, Serializer)} instead. */ @Deprecated public StreamsMetadata metadataForKey(final String storeName, @@ -1115,6 +1118,7 @@ public StreamsMetadata metadataForKey(final String storeName, * @param key type * @return {@link StreamsMetadata} for the {@code KafkaStreams} instance with the provided {@code storeName} and * {@code key} of this application or {@link StreamsMetadata#NOT_AVAILABLE} if Kafka Streams is (re-)initializing + * @deprecated Use {@link #queryMetadataForKey(String, Object, StreamPartitioner)} instead. */ @Deprecated public StreamsMetadata metadataForKey(final String storeName, @@ -1172,10 +1176,9 @@ public Map> allLocalStorePartitionLags() { // Obtain the current positions, of all the active-restoring and standby tasks for (final StreamThread streamThread : this.threads) { for (final StandbyTask standbyTask : streamThread.allStandbyTasks()) { - final Map changelogPartitionLimits = standbyTask.checkpointedOffsets(); + final Map checkpointedOffsets = standbyTask.checkpointedOffsets(); standbyTask.changelogPartitions().forEach(topicPartition -> - standbyChangelogPositions.put(topicPartition, - changelogPartitionLimits.getOrDefault(topicPartition, UNKNOWN_POSITION))); + standbyChangelogPositions.put(topicPartition, checkpointedOffsets.getOrDefault(topicPartition, UNKNOWN_POSITION))); } final Set restoringTaskIds = streamThread.restoringTaskIds(); @@ -1193,9 +1196,9 @@ public Map> allLocalStorePartitionLags() { } log.info("Current changelog positions, for active: " + activeChangelogPositions + " standby:" + standbyChangelogPositions); - final Map offsetSpecMap = new HashMap<>(); - Stream.concat(activeChangelogPositions.keySet().stream(), standbyChangelogPositions.keySet().stream()) - .forEach(topicPartition -> offsetSpecMap.put(topicPartition, OffsetSpec.latest())); + final Map offsetSpecMap = Stream.concat( + activeChangelogPositions.keySet().stream(), standbyChangelogPositions.keySet().stream()) + .collect(Collectors.toMap(Function.identity(), tp -> OffsetSpec.latest())); final Map allEndOffsets = new HashMap<>(); try { allEndOffsets.putAll(adminClient.listOffsets(offsetSpecMap).all().get()); diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java index 87520ed8a7e8f..20dd77ec82a47 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java @@ -34,7 +34,7 @@ public class KeyQueryMetadata { * Sentinel to indicate that the KeyQueryMetadata is currently unavailable. This can occur during rebalance * operations. */ - public final static KeyQueryMetadata NOT_AVAILABLE = new KeyQueryMetadata(new HostInfo("unavailable", -1), + public static final KeyQueryMetadata NOT_AVAILABLE = new KeyQueryMetadata(new HostInfo("unavailable", -1), Collections.emptySet(), -1); 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 e445a2c97223d..d6b2e97b48c57 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -305,7 +305,7 @@ public class StreamsConfig extends AbstractConfig { /**{@code application.server} */ @SuppressWarnings("WeakerAccess") public static final String APPLICATION_SERVER_CONFIG = "application.server"; - private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to an embedded user defined endpoint that can be used for state store discovery and interactive queries within a single KafkaStreams application"; + private static final String APPLICATION_SERVER_DOC = "A host:port pair pointing to a user-defined endpoint that can be used for state store discovery and interactive queries on this KafkaStreams instance."; /** {@code bootstrap.servers} */ @SuppressWarnings("WeakerAccess") 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 1f1e3cbabc080..9222ffd1546e3 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 @@ -50,7 +50,7 @@ public class StreamsMetadataState { private final Set globalStores; private final HostInfo thisHost; private Cluster clusterMetadata; - private StreamsMetadata myMetadata; + private StreamsMetadata localMetadata; public StreamsMetadataState(final InternalTopologyBuilder builder, final HostInfo thisHost) { this.builder = builder; @@ -78,8 +78,8 @@ public String toString(final String indent) { * * @return the {@link StreamsMetadata}s for the local instance in a {@link KafkaStreams} application */ - public synchronized StreamsMetadata getMyMetadata() { - return myMetadata; + public synchronized StreamsMetadata getLocalMetadata() { + return localMetadata; } /** @@ -130,12 +130,14 @@ public synchronized Collection getAllMetadataForStore(final Str * Note: the key may not exist in the {@link org.apache.kafka.streams.processor.StateStore}, * this method provides a way of finding which {@link StreamsMetadata} it would exist on. * + * * @param storeName Name of the store * @param key Key to use * @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 + * @deprecated Use {@link #getKeyQueryMetadataForKey(String, Object, Serializer)} instead. */ @Deprecated public synchronized StreamsMetadata getMetadataWithKey(final String storeName, @@ -155,7 +157,7 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam if (thisHost == UNKNOWN_HOST) { return allMetadata.get(0); } - return myMetadata; + return localMetadata; } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); @@ -223,7 +225,7 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String if (thisHost == UNKNOWN_HOST) { return new KeyQueryMetadata(allMetadata.get(0).hostInfo(), Collections.emptySet(), -1); } - return new KeyQueryMetadata(myMetadata.hostInfo(), Collections.emptySet(), -1); + return new KeyQueryMetadata(localMetadata.hostInfo(), Collections.emptySet(), -1); } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); @@ -245,6 +247,7 @@ public synchronized KeyQueryMetadata getKeyQueryMetadataForKey(final String * @param key type * @return The {@link StreamsMetadata} for the storeName and key or {@link StreamsMetadata#NOT_AVAILABLE} * if streams is (re-)initializing + * @deprecated Use {@link #getKeyQueryMetadataForKey(String, Object, StreamPartitioner)} instead. */ @Deprecated public synchronized StreamsMetadata getMetadataWithKey(final String storeName, @@ -259,12 +262,12 @@ public synchronized StreamsMetadata getMetadataWithKey(final String storeNam } if (globalStores.contains(storeName)) { - // global stores are on every node. if we dont' have the host info + // global stores are on every node. if we don't have the host info // for this host then just pick the first metadata if (thisHost == UNKNOWN_HOST) { return allMetadata.get(0); } - return myMetadata; + return localMetadata; } final SourceTopicsInfo sourceTopicsInfo = getSourceTopicsInfo(storeName); @@ -319,30 +322,30 @@ private void rebuildMetadata(final Map> activePart final Map> storeToSourceTopics = builder.stateStoreNameToSourceTopics(); Stream.concat(activePartitionHostMap.keySet().stream(), standbyPartitionHostMap.keySet().stream()) .distinct() - .forEach(key -> { + .forEach(hostInfo -> { final Set activePartitionsOnHost = new HashSet<>(); final Set activeStoresOnHost = new HashSet<>(); - if (activePartitionHostMap.containsKey(key)) { - activePartitionsOnHost.addAll(activePartitionHostMap.get(key)); + if (activePartitionHostMap.containsKey(hostInfo)) { + activePartitionsOnHost.addAll(activePartitionHostMap.get(hostInfo)); activeStoresOnHost.addAll(getStoresOnHost(storeToSourceTopics, activePartitionsOnHost)); } activeStoresOnHost.addAll(globalStores); final Set standbyPartitionsOnHost = new HashSet<>(); final Set standbyStoresOnHost = new HashSet<>(); - if (standbyPartitionHostMap.containsKey(key)) { - standbyPartitionsOnHost.addAll(standbyPartitionHostMap.get(key)); + if (standbyPartitionHostMap.containsKey(hostInfo)) { + standbyPartitionsOnHost.addAll(standbyPartitionHostMap.get(hostInfo)); standbyStoresOnHost.addAll(getStoresOnHost(storeToSourceTopics, standbyPartitionsOnHost)); } - final StreamsMetadata metadata = new StreamsMetadata(key, + final StreamsMetadata metadata = new StreamsMetadata(hostInfo, activeStoresOnHost, activePartitionsOnHost, standbyStoresOnHost, standbyPartitionsOnHost); allMetadata.add(metadata); - if (key.equals(thisHost)) { - myMetadata = metadata; + if (hostInfo.equals(thisHost)) { + localMetadata = metadata; } }); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index 5329e2cfd6d2f..f7d8541c0326b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -16,8 +16,13 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; -import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.utils.ByteBufferInputStream; +import org.apache.kafka.streams.errors.TaskAssignmentException; +import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.state.HostInfo; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.io.ByteArrayOutputStream; import java.io.DataInputStream; @@ -34,13 +39,9 @@ import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.utils.ByteBufferInputStream; -import org.apache.kafka.streams.errors.TaskAssignmentException; -import org.apache.kafka.streams.processor.TaskId; -import org.apache.kafka.streams.state.HostInfo; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; + +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN; public class AssignmentInfo { private static final Logger log = LoggerFactory.getLogger(AssignmentInfo.class); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java index 0c6781cb50e63..50c2d6837fd2d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StreamsMetadata.java @@ -42,23 +42,14 @@ public class StreamsMetadata { Collections.emptySet()); private final HostInfo hostInfo; - /** - * State stores owned by the instance as an active replica - */ + private final Set stateStoreNames; - /** - * Topic partitions consumed by the instance as an active replica - */ + private final Set topicPartitions; - /** - * State stores owned by the instance as a standby replica - */ + private final Set standbyStateStoreNames; - /** - * (Source) Topic partitions for which the instance acts as standby. - */ - private final Set standbyTopicPartitions; + private final Set standbyTopicPartitions; public StreamsMetadata(final HostInfo hostInfo, final Set stateStoreNames, @@ -73,26 +64,52 @@ public StreamsMetadata(final HostInfo hostInfo, this.standbyStateStoreNames = standbyStateStoreNames; } - public Set standbyTopicPartitions() { - return standbyTopicPartitions; - } - - public Set standbyStateStoreNames() { - return standbyStateStoreNames; - } - + /** + * The value of {@link org.apache.kafka.streams.StreamsConfig#APPLICATION_SERVER_CONFIG} configured for the streams + * instance, which is typically host/port + * + * @return {@link HostInfo} corresponding to the streams instance + */ public HostInfo hostInfo() { return hostInfo; } + /** + * State stores owned by the instance as an active replica + * + * @return set of active state store names + */ public Set stateStoreNames() { return stateStoreNames; } + /** + * Topic partitions consumed by the instance as an active replica + * + * @return set of active topic partitions + */ public Set topicPartitions() { return topicPartitions; } + /** + * (Source) Topic partitions for which the instance acts as standby. + * + * @return set of standby topic partitions + */ + public Set standbyTopicPartitions() { + return standbyTopicPartitions; + } + + /** + * State stores owned by the instance as a standby replica + * + * @return set of standby state store names + */ + public Set standbyStateStoreNames() { + return standbyStateStoreNames; + } + public String host() { return hostInfo.host(); } @@ -110,21 +127,13 @@ public boolean equals(final Object o) { if (o == null || getClass() != o.getClass()) { return false; } - final StreamsMetadata that = (StreamsMetadata) o; - if (!hostInfo.equals(that.hostInfo)) { - return false; - } - if (!stateStoreNames.equals(that.stateStoreNames)) { - return false; - } - if (!topicPartitions.equals(that.topicPartitions)) { - return false; - } - if (!standbyStateStoreNames.equals(that.standbyStateStoreNames)) { - return false; - } - return standbyTopicPartitions.equals(that.standbyTopicPartitions); + final StreamsMetadata that = (StreamsMetadata) o; + return Objects.equals(hostInfo, that.hostInfo) + && Objects.equals(stateStoreNames, that.stateStoreNames) + && Objects.equals(topicPartitions, that.topicPartitions) + && Objects.equals(standbyStateStoreNames, that.standbyStateStoreNames) + && Objects.equals(standbyTopicPartitions, that.standbyTopicPartitions); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java index 478d9e37ec6a7..7c2b472b4fcc7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StreamThreadStateStoreProvider.java @@ -50,7 +50,7 @@ public List stores(final String storeName, final QueryableStoreType qu } if (!streamThread.isRunning()) { throw new InvalidStateStoreException("Cannot get state store " + storeName + " because the stream thread is " + - streamThread.state() + ", not RUNNING"); + streamThread.state() + ", not RUNNING or REBALANCING"); } final List stores = new ArrayList<>(); final Set tasks = new HashSet<>(streamThread.tasks().values()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 6fd18172c182f..7f92ee3cc9c48 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -126,7 +126,9 @@ public void before() { @Test public void shouldNotThrowExceptionWhenOnChangeNotCalled() { - new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne).getAllMetadataForStore("store"); + final Collection metadata = new StreamsMetadataState( + TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne).getAllMetadataForStore("store"); + assertEquals(0, metadata.size()); } @Test diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java index c7ed534af79c8..9206969e799d0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfoTest.java @@ -24,11 +24,13 @@ import java.util.Arrays; import java.util.Collections; -import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.UNKNOWN; import static org.junit.Assert.assertEquals; @@ -39,36 +41,31 @@ public class AssignmentInfoTest { new TaskId(0, 1), new TaskId(1, 0), new TaskId(1, 1)); - private final Map> standbyTasks = new HashMap>() { - { - put(new TaskId(1, 0), - Utils.mkSet(new TopicPartition("t1", 0), new TopicPartition("t2", 0))); - put(new TaskId(1, 1), - Utils.mkSet(new TopicPartition("t1", 1), new TopicPartition("t2", 1))); - } - }; - private final Map> activeAssignment = new HashMap>() { - { - put(new HostInfo("localhost", 8088), Utils.mkSet( - new TopicPartition("t0", 0), - new TopicPartition("t1", 0), - new TopicPartition("t2", 0))); - put(new HostInfo("localhost", 8089), Utils.mkSet( - new TopicPartition("t0", 1), - new TopicPartition("t1", 1), - new TopicPartition("t2", 1))); - } - }; - private final Map> standbyAssignment = new HashMap>() { - { - put(new HostInfo("localhost", 8088), Utils.mkSet( + + private final Map> standbyTasks = mkMap( + mkEntry(new TaskId(1, 0), mkSet(new TopicPartition("t1", 0), new TopicPartition("t2", 0))), + mkEntry(new TaskId(1, 1), mkSet(new TopicPartition("t1", 1), new TopicPartition("t2", 1))) + ); + + private final Map> activeAssignment = mkMap( + mkEntry(new HostInfo("localhost", 8088), + mkSet(new TopicPartition("t0", 0), new TopicPartition("t1", 0), - new TopicPartition("t2", 0))); - put(new HostInfo("localhost", 8089), Utils.mkSet( + new TopicPartition("t2", 0))), + mkEntry(new HostInfo("localhost", 8089), + mkSet(new TopicPartition("t0", 1), new TopicPartition("t1", 1), - new TopicPartition("t2", 1))); - } - }; + new TopicPartition("t2", 1))) + ); + + private final Map> standbyAssignment = mkMap( + mkEntry(new HostInfo("localhost", 8088), + Utils.mkSet(new TopicPartition("t1", 0), + new TopicPartition("t2", 0))), + mkEntry(new HostInfo("localhost", 8089), + Utils.mkSet(new TopicPartition("t1", 1), + new TopicPartition("t2", 1))) + ); @Test public void shouldUseLatestSupportedVersionByDefault() { From e38fa9adcfdb94210689b3e16fd18294146797ef Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Thu, 9 Jan 2020 07:44:12 -0800 Subject: [PATCH 19/21] Bringing KafkaStreams#metadataForKey() back into QueryableStateIntegrationTest + few cleanups --- .../kafka/streams/KeyQueryMetadata.java | 6 +- .../processor/internals/StreamThread.java | 4 + .../QueryableStateIntegrationTest.java | 32 +++++--- .../internals/StreamsMetadataStateTest.java | 80 +++++++++++-------- 4 files changed, 78 insertions(+), 44 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java index 20dd77ec82a47..26b35fe324329 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyQueryMetadata.java @@ -83,12 +83,14 @@ public boolean equals(final Object obj) { return false; } final KeyQueryMetadata keyQueryMetadata = (KeyQueryMetadata) obj; - return Objects.equals(keyQueryMetadata.activeHost, activeHost) && Objects.equals(keyQueryMetadata.standbyHosts, standbyHosts) && Objects.equals(keyQueryMetadata.partition, partition); + return Objects.equals(keyQueryMetadata.activeHost, activeHost) + && Objects.equals(keyQueryMetadata.standbyHosts, standbyHosts) + && Objects.equals(keyQueryMetadata.partition, partition); } @Override public String toString() { - return "KeyQueryMetadata{" + + return "KeyQueryMetadata {" + "activeHost=" + activeHost + ", standbyHosts=" + standbyHosts + ", partition=" + partition + 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 399dc0eda588c..23a411e770de2 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 @@ -1276,4 +1276,8 @@ Map>> standbyRecords() { int currentNumIterations() { return numIterations; } + + public StreamThread.StateListener stateListener() { + return stateListener; + } } 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 8a7d7c4c16f1a..3127893351778 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 @@ -93,6 +93,7 @@ import org.apache.kafka.streams.state.QueryableStoreTypes; import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; 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.test.IntegrationTest; import org.apache.kafka.test.MockMapper; @@ -275,6 +276,7 @@ private void verifyOffsetLagFetch(final List streamsList, } } + @SuppressWarnings("deprecation") private void verifyAllKVKeys(final List streamsList, final KafkaStreams streams, final KafkaStreamsTest.StateListenerStub stateListener, @@ -287,17 +289,22 @@ private void verifyAllKVKeys(final List streamsList, final List nullStoreKeys = new ArrayList<>(); final List nullValueKeys = new ArrayList<>(); final Map exceptionalKeys = new TreeMap<>(); + final StringSerializer serializer = new StringSerializer(); for (final String key: keys) { try { - final KeyQueryMetadata metadata = streams - .queryMetadataForKey(storeName, key, new StringSerializer()); - if (metadata == null || metadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { + final KeyQueryMetadata queryMetadata = streams.queryMetadataForKey(storeName, key, serializer); + final StreamsMetadata metadata = streams.metadataForKey(storeName, key, serializer); + if (queryMetadata == null || queryMetadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { noMetadataKeys.add(key); continue; } + assertThat(metadata.hostInfo(), equalTo(queryMetadata.getActiveHost())); + if (!pickInstanceByPort) { + assertThat("Should have standbys to query from", queryMetadata.getStandbyHosts().size() > 0); + } - final int index = metadata.getActiveHost().port(); + final int index = queryMetadata.getActiveHost().port(); final KafkaStreams streamsWithKey = pickInstanceByPort ? streamsList.get(index) : streams; final ReadOnlyKeyValueStore store = streamsWithKey.store(storeName, QueryableStoreTypes.keyValueStore()); @@ -308,7 +315,6 @@ private void verifyAllKVKeys(final List streamsList, if (store.get(key) == null) { nullValueKeys.add(key); - continue; } } catch (final InvalidStateStoreException e) { if (stateListener.mapStates.get(KafkaStreams.State.REBALANCING) < 1) { @@ -325,6 +331,7 @@ private void verifyAllKVKeys(final List streamsList, }); } + @SuppressWarnings("deprecation") private void verifyAllWindowedKeys(final List streamsList, final KafkaStreams streams, final KafkaStreamsTest.StateListenerStub stateListenerStub, @@ -339,17 +346,24 @@ private void verifyAllWindowedKeys(final List streamsList, final List nullStoreKeys = new ArrayList<>(); final List nullValueKeys = new ArrayList<>(); final Map exceptionalKeys = new TreeMap<>(); + final StringSerializer serializer = new StringSerializer(); for (final String key: keys) { try { - final KeyQueryMetadata metadata = streams - .queryMetadataForKey(storeName, key, new StringSerializer()); - if (metadata == null || metadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { + final KeyQueryMetadata queryMetadata = streams.queryMetadataForKey(storeName, key, serializer); + final StreamsMetadata metadata = streams.metadataForKey(storeName, key, serializer); + if (queryMetadata == null || queryMetadata.equals(KeyQueryMetadata.NOT_AVAILABLE)) { noMetadataKeys.add(key); continue; } + assertThat(metadata.hostInfo(), equalTo(queryMetadata.getActiveHost())); + if (pickInstanceByPort) { + assertThat(queryMetadata.getStandbyHosts().size(), equalTo(0)); + } else { + assertThat("Should have standbys to query from", queryMetadata.getStandbyHosts().size() > 0); + } - final int index = metadata.getActiveHost().port(); + final int index = queryMetadata.getActiveHost().port(); final KafkaStreams streamsWithKey = pickInstanceByPort ? streamsList.get(index) : streams; final ReadOnlyWindowStore store = streamsWithKey.store(storeName, QueryableStoreTypes.windowStore()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java index 7f92ee3cc9c48..6a5c26aa6be7a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamsMetadataStateTest.java @@ -16,6 +16,7 @@ */ package org.apache.kafka.streams.processor.internals; +import static org.apache.kafka.common.utils.Utils.mkSet; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; @@ -36,7 +37,6 @@ import org.apache.kafka.common.TopicPartition; 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.KeyQueryMetadata; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.TopologyWrapper; @@ -67,6 +67,7 @@ public class StreamsMetadataStateTest { private Cluster cluster; private final String globalTable = "global-table"; private StreamPartitioner partitioner; + private Set storeNames; @Before public void before() { @@ -102,12 +103,12 @@ public void before() { hostTwo = new HostInfo("host-two", 9090); hostThree = new HostInfo("host-three", 7070); hostToActivePartitions = new HashMap<>(); - hostToActivePartitions.put(hostOne, Utils.mkSet(topic1P0, topic2P1, topic4P0)); - hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1)); + hostToActivePartitions.put(hostOne, mkSet(topic1P0, topic2P1, topic4P0)); + hostToActivePartitions.put(hostTwo, mkSet(topic2P0, topic1P1)); hostToActivePartitions.put(hostThree, Collections.singleton(topic3P0)); hostToStandbyPartitions = new HashMap<>(); - hostToStandbyPartitions.put(hostThree, Utils.mkSet(topic1P0, topic2P1, topic4P0)); - hostToStandbyPartitions.put(hostOne, Utils.mkSet(topic2P0, topic1P1)); + hostToStandbyPartitions.put(hostThree, mkSet(topic1P0, topic2P1, topic4P0)); + hostToStandbyPartitions.put(hostOne, mkSet(topic2P0, topic1P1)); hostToStandbyPartitions.put(hostTwo, Collections.singleton(topic3P0)); final List partitionInfos = Arrays.asList( @@ -122,6 +123,7 @@ public void before() { metadataState = new StreamsMetadataState(TopologyWrapper.getInternalTopologyBuilder(builder.build()), hostOne); metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster); partitioner = (topic, key, value, numPartitions) -> 1; + storeNames = mkSet("table-one", "table-two", "merged-table", globalTable); } @Test @@ -134,20 +136,20 @@ public void shouldNotThrowExceptionWhenOnChangeNotCalled() { @Test public void shouldGetAllStreamInstances() { final StreamsMetadata one = new StreamsMetadata(hostOne, - Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), - Utils.mkSet("table-one", "table-two", "merged-table"), - Utils.mkSet(topic2P0, topic1P1)); + mkSet(globalTable, "table-one", "table-two", "merged-table"), + mkSet(topic1P0, topic2P1, topic4P0), + mkSet("table-one", "table-two", "merged-table"), + mkSet(topic2P0, topic1P1)); final StreamsMetadata two = new StreamsMetadata(hostTwo, - Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), - Utils.mkSet("table-three"), - Utils.mkSet(topic3P0)); + mkSet(globalTable, "table-two", "table-one", "merged-table"), + mkSet(topic2P0, topic1P1), + mkSet("table-three"), + mkSet(topic3P0)); final StreamsMetadata three = new StreamsMetadata(hostThree, - Utils.mkSet(globalTable, "table-three"), + mkSet(globalTable, "table-three"), Collections.singleton(topic3P0), - Utils.mkSet("table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0)); + mkSet("table-one", "table-two", "merged-table"), + mkSet(topic1P0, topic2P1, topic4P0)); final Collection actual = metadataState.getAllMetadata(); assertEquals(3, actual.size()); @@ -162,7 +164,7 @@ public void shouldGetAllStreamsInstancesWithNoStores() { final TopicPartition tp5 = new TopicPartition("topic-five", 1); final HostInfo hostFour = new HostInfo("host-four", 8080); - hostToActivePartitions.put(hostFour, Utils.mkSet(tp5)); + hostToActivePartitions.put(hostFour, mkSet(tp5)); metadataState.onChange(hostToActivePartitions, Collections.emptyMap(), cluster.withPartitions(Collections.singletonMap(tp5, new PartitionInfo("topic-five", 1, null, null, null)))); @@ -176,15 +178,15 @@ public void shouldGetAllStreamsInstancesWithNoStores() { @Test public void shouldGetInstancesForStoreName() { final StreamsMetadata one = new StreamsMetadata(hostOne, - Utils.mkSet(globalTable, "table-one", "table-two", "merged-table"), - Utils.mkSet(topic1P0, topic2P1, topic4P0), - Utils.mkSet("table-one", "table-two", "merged-table"), - Utils.mkSet(topic2P0, topic1P1)); + mkSet(globalTable, "table-one", "table-two", "merged-table"), + mkSet(topic1P0, topic2P1, topic4P0), + mkSet("table-one", "table-two", "merged-table"), + mkSet(topic2P0, topic1P1)); final StreamsMetadata two = new StreamsMetadata(hostTwo, - Utils.mkSet(globalTable, "table-two", "table-one", "merged-table"), - Utils.mkSet(topic2P0, topic1P1), - Utils.mkSet("table-three"), - Utils.mkSet(topic3P0)); + mkSet(globalTable, "table-two", "table-one", "merged-table"), + mkSet(topic2P0, topic1P1), + mkSet("table-three"), + mkSet(topic3P0)); final Collection actual = metadataState.getAllMetadataForStore("table-one"); final Map actualAsMap = actual.stream() .collect(Collectors.toMap(StreamsMetadata::hostInfo, Function.identity())); @@ -209,12 +211,12 @@ public void shouldReturnEmptyCollectionOnGetAllInstancesWithStoreWhenStoreDoesnt @Test public void shouldGetInstanceWithKey() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); - hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); + hostToActivePartitions.put(hostTwo, mkSet(topic2P0, tp4)); metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); - final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, Utils.mkSet(hostTwo), 0); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostThree, mkSet(hostTwo), 0); final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("table-three", "the-key", Serdes.String().serializer()); @@ -224,7 +226,7 @@ public void shouldGetInstanceWithKey() { @Test public void shouldGetInstanceWithKeyAndCustomPartitioner() { final TopicPartition tp4 = new TopicPartition("topic-three", 1); - hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, tp4)); + hostToActivePartitions.put(hostTwo, mkSet(topic2P0, tp4)); metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(tp4, new PartitionInfo("topic-three", 1, null, null, null)))); @@ -248,12 +250,12 @@ public void shouldReturnNotAvailableWhenClusterIsEmpty() { @Test public void shouldGetInstanceWithKeyWithMergedStreams() { final TopicPartition topic2P2 = new TopicPartition("topic-two", 2); - hostToActivePartitions.put(hostTwo, Utils.mkSet(topic2P0, topic1P1, topic2P2)); - hostToStandbyPartitions.put(hostOne, Utils.mkSet(topic2P0, topic1P1, topic2P2)); + hostToActivePartitions.put(hostTwo, mkSet(topic2P0, topic1P1, topic2P2)); + hostToStandbyPartitions.put(hostOne, mkSet(topic2P0, topic1P1, topic2P2)); metadataState.onChange(hostToActivePartitions, hostToStandbyPartitions, cluster.withPartitions(Collections.singletonMap(topic2P2, new PartitionInfo("topic-two", 2, null, null, null)))); - final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, Utils.mkSet(hostOne), 2); + final KeyQueryMetadata expected = new KeyQueryMetadata(hostTwo, mkSet(hostOne), 2); final KeyQueryMetadata actual = metadataState.getKeyQueryMetadataForKey("merged-table", "the-key", (topic, key, value, numPartitions) -> 2); @@ -300,9 +302,20 @@ public void shouldHaveGlobalStoreInAllMetadata() { } @Test - public void shouldGetMyMetadataForGlobalStoreWithKey() { + public void shouldGetLocalMetadataWithRightActiveStandbyInfo() { + assertEquals(hostOne, metadataState.getLocalMetadata().hostInfo()); + assertEquals(hostToActivePartitions.get(hostOne), metadataState.getLocalMetadata().topicPartitions()); + assertEquals(hostToStandbyPartitions.get(hostOne), metadataState.getLocalMetadata().standbyTopicPartitions()); + assertEquals(storeNames, metadataState.getLocalMetadata().stateStoreNames()); + assertEquals(storeNames.stream().filter(s -> !s.equals(globalTable)).collect(Collectors.toSet()), + metadataState.getLocalMetadata().standbyStateStoreNames()); + } + + @Test + public void shouldGetQueryMetadataForGlobalStoreWithKey() { final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataForKey(globalTable, "key", Serdes.String().serializer()); assertEquals(hostOne, metadata.getActiveHost()); + assertTrue(metadata.getStandbyHosts().isEmpty()); } @Test @@ -313,9 +326,10 @@ public void shouldGetAnyHostForGlobalStoreByKeyIfMyHostUnknown() { } @Test - public void shouldGetMyMetadataForGlobalStoreWithKeyAndPartitioner() { + public void shouldGetQueryMetadataForGlobalStoreWithKeyAndPartitioner() { final KeyQueryMetadata metadata = metadataState.getKeyQueryMetadataForKey(globalTable, "key", partitioner); assertEquals(hostOne, metadata.getActiveHost()); + assertTrue(metadata.getStandbyHosts().isEmpty()); } @Test From 1cfa5c5429e6e0299e5cd19a06817c18c12383a1 Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Fri, 10 Jan 2020 17:52:26 -0800 Subject: [PATCH 20/21] Adding LagFetchIntegrationTest to test the lag values obtained during rebalancing/restoration --- .../integration/LagFetchIntegrationTest.java | 309 ++++++++++++++++++ .../QueryableStateIntegrationTest.java | 1 - 2 files changed, 309 insertions(+), 1 deletion(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java new file mode 100644 index 0000000000000..bfc19c34c734e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java @@ -0,0 +1,309 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.integration; + +import static org.apache.kafka.common.utils.Utils.mkSet; +import static org.apache.kafka.streams.integration.utils.IntegrationTestUtils.startApplicationAndWaitUntilRunning; +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.IsEqual.equalTo; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.nio.file.Files; +import java.nio.file.Path; +import java.time.Duration; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.CyclicBarrier; +import java.util.concurrent.TimeUnit; +import kafka.utils.MockTime; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +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.KafkaStreamsWrapper; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.LagInfo; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.processor.StateRestoreListener; +import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; + +@Category({IntegrationTest.class}) +public class LagFetchIntegrationTest { + + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + private static final long CONSUMER_TIMEOUT_MS = 60000; + + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + private final MockTime mockTime = CLUSTER.time; + private Properties streamsConfiguration; + private Properties consumerConfiguration; + private String inputTopicName; + private String outputTopicName; + private String stateStoreName; + + @Rule + public TestName name = new TestName(); + + @Before + public void before() { + inputTopicName = "input-topic-" + name.getMethodName(); + outputTopicName = "output-topic-" + name.getMethodName(); + stateStoreName = "lagfetch-test-store" + name.getMethodName(); + + streamsConfiguration = new Properties(); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "lag-fetch-" + name.getMethodName()); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + + consumerConfiguration = new Properties(); + consumerConfiguration.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + consumerConfiguration.setProperty(ConsumerConfig.GROUP_ID_CONFIG, name.getMethodName() + "-consumer"); + consumerConfiguration.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + consumerConfiguration.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + consumerConfiguration.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class.getName()); + } + + @After + public void shutdown() throws Exception { + IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); + } + + @Test + public void shouldBeAbleToFetchValidLagsDuringRebalancing() throws Exception { + final CountDownLatch latchTillActiveIsRunning = new CountDownLatch(1); + final CountDownLatch latchTillStandbyIsRunning = new CountDownLatch(1); + final CountDownLatch latchTillStandbyHasPartitionsAssigned = new CountDownLatch(1); + final CyclicBarrier lagCheckBarrier = new CyclicBarrier(2); + final List streamsList = new ArrayList<>(); + + IntegrationTestUtils.produceKeyValuesSynchronously( + inputTopicName, + mkSet(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + StringSerializer.class, + LongSerializer.class, + new Properties()), + mockTime); + + // create stream threads + for (int i = 0; i < 2; i++) { + final Properties props = (Properties) streamsConfiguration.clone(); + final File stateDir = folder.newFolder("state-" + i); + props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:" + i); + props.put(StreamsConfig.CLIENT_ID_CONFIG, "instance-" + i); + props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.getAbsolutePath()); + + final StreamsBuilder builder = new StreamsBuilder(); + final KTable t1 = builder.table(inputTopicName, Materialized.as(stateStoreName)); + t1.toStream().to(outputTopicName); + final KafkaStreamsWrapper streams = new KafkaStreamsWrapper(builder.build(), props); + streamsList.add(streams); + } + + final KafkaStreamsWrapper activeStreams = streamsList.get(0); + final KafkaStreamsWrapper standbyStreams = streamsList.get(1); + activeStreams.setStreamThreadStateListener((thread, newState, oldState) -> { + if (newState == StreamThread.State.RUNNING) { + latchTillActiveIsRunning.countDown(); + } + }); + standbyStreams.setStreamThreadStateListener((thread, newState, oldState) -> { + if (oldState == StreamThread.State.PARTITIONS_ASSIGNED && newState == StreamThread.State.RUNNING) { + latchTillStandbyHasPartitionsAssigned.countDown(); + try { + lagCheckBarrier.await(60, TimeUnit.SECONDS); + } catch (final Exception e) { + throw new RuntimeException(e); + } + } else if (newState == StreamThread.State.RUNNING) { + latchTillStandbyIsRunning.countDown(); + } + }); + + try { + // First start up the active. + Map> offsetLagInfoMap = activeStreams.allLocalStorePartitionLags(); + assertThat(offsetLagInfoMap.size(), equalTo(0)); + activeStreams.start(); + latchTillActiveIsRunning.await(60, TimeUnit.SECONDS); + + IntegrationTestUtils.waitUntilMinValuesRecordsReceived( + consumerConfiguration, + outputTopicName, + 5, + CONSUMER_TIMEOUT_MS); + // Check the active reports proper lag values. + offsetLagInfoMap = activeStreams.allLocalStorePartitionLags(); + assertThat(offsetLagInfoMap.size(), equalTo(1)); + assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName))); + assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1)); + LagInfo lagInfo = offsetLagInfoMap.get(stateStoreName).get(0); + assertThat(lagInfo.currentOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.offsetLag(), equalTo(0L)); + + // start up the standby & make it pause right after it has partition assigned + standbyStreams.start(); + latchTillStandbyHasPartitionsAssigned.await(60, TimeUnit.SECONDS); + offsetLagInfoMap = standbyStreams.allLocalStorePartitionLags(); + assertThat(offsetLagInfoMap.size(), equalTo(1)); + assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName))); + assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1)); + lagInfo = offsetLagInfoMap.get(stateStoreName).get(0); + assertThat(lagInfo.currentOffsetPosition(), equalTo(0L)); + assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.offsetLag(), equalTo(5L)); + // standby thread wont proceed to RUNNING before this barrier is crossed + lagCheckBarrier.await(60, TimeUnit.SECONDS); + + // wait till the lag goes down to 0, on the standby + TestUtils.waitForCondition(() -> standbyStreams.allLocalStorePartitionLags().get(stateStoreName).get(0).offsetLag() == 0, + "Standby should eventually catchup and have zero lag."); + } finally { + for (final KafkaStreams streams : streamsList) { + streams.close(); + } + } + } + + @Test + public void shouldBeAbleToFetchValidLagsDuringRestoration() throws Exception { + IntegrationTestUtils.produceKeyValuesSynchronously( + inputTopicName, + mkSet(new KeyValue<>("k1", 1L), new KeyValue<>("k2", 2L), new KeyValue<>("k3", 3L), new KeyValue<>("k4", 4L), new KeyValue<>("k5", 5L)), + TestUtils.producerConfig( + CLUSTER.bootstrapServers(), + StringSerializer.class, + LongSerializer.class, + new Properties()), + mockTime); + + // create stream threads + final Properties props = (Properties) streamsConfiguration.clone(); + final File stateDir = folder.newFolder("state"); + props.put(StreamsConfig.APPLICATION_SERVER_CONFIG, "localhost:0"); + props.put(StreamsConfig.CLIENT_ID_CONFIG, "instance-0"); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.getAbsolutePath()); + + final StreamsBuilder builder = new StreamsBuilder(); + final KTable t1 = builder.table(inputTopicName, Materialized.as(stateStoreName)); + t1.toStream().to(outputTopicName); + final KafkaStreams streams = new KafkaStreams(builder.build(), props); + + try { + // First start up the active. + Map> offsetLagInfoMap = streams.allLocalStorePartitionLags(); + assertThat(offsetLagInfoMap.size(), equalTo(0)); + + // Get the instance to fully catch up and reach RUNNING state + startApplicationAndWaitUntilRunning(Collections.singletonList(streams), Duration.ofSeconds(60)); + IntegrationTestUtils.waitUntilMinValuesRecordsReceived( + consumerConfiguration, + outputTopicName, + 5, + CONSUMER_TIMEOUT_MS); + + // check for proper lag values. + offsetLagInfoMap = streams.allLocalStorePartitionLags(); + assertThat(offsetLagInfoMap.size(), equalTo(1)); + assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName))); + assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1)); + LagInfo lagInfo = offsetLagInfoMap.get(stateStoreName).get(0); + assertThat(lagInfo.currentOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.offsetLag(), equalTo(0L)); + + // Kill instance, delete state to force restoration. + assertThat("Streams instance did not close within timeout", streams.close(Duration.ofSeconds(60))); + IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); + Files.walk(stateDir.toPath()).sorted(Comparator.reverseOrder()) + .map(Path::toFile) + .forEach(f -> assertTrue("Some state " + f + " could not be deleted", f.delete())); + + // wait till the lag goes down to 0 + final KafkaStreams restartedStreams = new KafkaStreams(builder.build(), props); + // set a state restoration listener to track progress of restoration + final Map> restoreStartLagInfo = new HashMap<>(); + final Map> restoreEndLagInfo = new HashMap<>(); + restartedStreams.setGlobalStateRestoreListener(new StateRestoreListener() { + @Override + public void onRestoreStart(final TopicPartition topicPartition, final String storeName, final long startingOffset, final long endingOffset) { + restoreStartLagInfo.putAll(restartedStreams.allLocalStorePartitionLags()); + } + + @Override + public void onBatchRestored(final TopicPartition topicPartition, final String storeName, final long batchEndOffset, final long numRestored) { + } + + @Override + public void onRestoreEnd(final TopicPartition topicPartition, final String storeName, final long totalRestored) { + restoreEndLagInfo.putAll(restartedStreams.allLocalStorePartitionLags()); + } + }); + + restartedStreams.start(); + TestUtils.waitForCondition(() -> restartedStreams.allLocalStorePartitionLags().get(stateStoreName).get(0).offsetLag() == 0, + "Standby should eventually catchup and have zero lag."); + lagInfo = restoreStartLagInfo.get(stateStoreName).get(0); + assertThat(lagInfo.currentOffsetPosition(), equalTo(0L)); + assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.offsetLag(), equalTo(5L)); + + lagInfo = restoreEndLagInfo.get(stateStoreName).get(0); + assertThat(lagInfo.currentOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(lagInfo.offsetLag(), equalTo(0L)); + + } finally { + streams.close(); + } + } +} 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 3127893351778..bd6d08fe1cdb5 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 @@ -374,7 +374,6 @@ private void verifyAllWindowedKeys(final List streamsList, if (store.fetch(key, ofEpochMilli(from), ofEpochMilli(to)) == null) { nullValueKeys.add(key); - continue; } } catch (final InvalidStateStoreException e) { // there must have been at least one rebalance state From 98879d79f5cc95e3c85ad52d1c438c31dfb69b2f Mon Sep 17 00:00:00 2001 From: vinoth chandar Date: Mon, 13 Jan 2020 10:46:45 -0800 Subject: [PATCH 21/21] Small change to enable LagInfo#equals() for test coverage --- .../integration/LagFetchIntegrationTest.java | 22 ++++++++----------- 1 file changed, 9 insertions(+), 13 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java index bfc19c34c734e..9afd6fcb32110 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/LagFetchIntegrationTest.java @@ -256,10 +256,10 @@ public void shouldBeAbleToFetchValidLagsDuringRestoration() throws Exception { assertThat(offsetLagInfoMap.size(), equalTo(1)); assertThat(offsetLagInfoMap.keySet(), equalTo(mkSet(stateStoreName))); assertThat(offsetLagInfoMap.get(stateStoreName).size(), equalTo(1)); - LagInfo lagInfo = offsetLagInfoMap.get(stateStoreName).get(0); - assertThat(lagInfo.currentOffsetPosition(), equalTo(5L)); - assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); - assertThat(lagInfo.offsetLag(), equalTo(0L)); + final LagInfo zeroLagInfo = offsetLagInfoMap.get(stateStoreName).get(0); + assertThat(zeroLagInfo.currentOffsetPosition(), equalTo(5L)); + assertThat(zeroLagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(zeroLagInfo.offsetLag(), equalTo(0L)); // Kill instance, delete state to force restoration. assertThat("Streams instance did not close within timeout", streams.close(Duration.ofSeconds(60))); @@ -292,16 +292,12 @@ public void onRestoreEnd(final TopicPartition topicPartition, final String store restartedStreams.start(); TestUtils.waitForCondition(() -> restartedStreams.allLocalStorePartitionLags().get(stateStoreName).get(0).offsetLag() == 0, "Standby should eventually catchup and have zero lag."); - lagInfo = restoreStartLagInfo.get(stateStoreName).get(0); - assertThat(lagInfo.currentOffsetPosition(), equalTo(0L)); - assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); - assertThat(lagInfo.offsetLag(), equalTo(5L)); - - lagInfo = restoreEndLagInfo.get(stateStoreName).get(0); - assertThat(lagInfo.currentOffsetPosition(), equalTo(5L)); - assertThat(lagInfo.endOffsetPosition(), equalTo(5L)); - assertThat(lagInfo.offsetLag(), equalTo(0L)); + final LagInfo fullLagInfo = restoreStartLagInfo.get(stateStoreName).get(0); + assertThat(fullLagInfo.currentOffsetPosition(), equalTo(0L)); + assertThat(fullLagInfo.endOffsetPosition(), equalTo(5L)); + assertThat(fullLagInfo.offsetLag(), equalTo(5L)); + assertThat(zeroLagInfo, equalTo(restoreEndLagInfo.get(stateStoreName).get(0))); } finally { streams.close(); }