From 5a67b592c6947c20f07f116037cc1ed159a06d50 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Thu, 13 Feb 2020 20:46:15 -0800 Subject: [PATCH 01/24] use taskLags to infer prev/standby tasks if necessary --- .../internals/StreamsPartitionAssignor.java | 13 +++-- .../StreamsAssignmentProtocolVersions.java | 2 +- .../assignment/SubscriptionInfo.java | 58 +++++++++++++++++-- .../common/message/SubscriptionInfo.json | 32 +++++++++- 4 files changed, 91 insertions(+), 14 deletions(-) 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 abaebcb808f45..da4795767c2eb 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 @@ -224,14 +224,19 @@ public ByteBuffer subscriptionUserData(final Set topics) { topics, standbyTasks, rebalanceProtocol); + + final Map taskLags = new HashMap<>(activeTasks.stream() + .collect(Collectors.toMap(t -> t, l -> -1))); + taskLags.putAll(standbyTasks.stream() + .collect(Collectors.toMap(t -> t, l -> 0))); + return new SubscriptionInfo( usedSubscriptionMetadataVersion, LATEST_SUPPORTED_VERSION, taskManager.processId(), - activeTasks, - standbyTasks, - userEndPoint) - .encode(); + userEndPoint, + taskLags) + .encode(); } protected static Set prepareForSubscription(final TaskManager taskManager, 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 f091907361644..51d70a8a7fd39 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 = 6; + public static final int LATEST_SUPPORTED_VERSION = 7; private StreamsAssignmentProtocolVersions() {} } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index b410a0dfa059a..64c2dbc08dd33 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.HashSet; +import java.util.Map; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.streams.errors.TaskAssignmentException; @@ -41,6 +43,7 @@ public class SubscriptionInfo { private final SubscriptionInfoData data; private Set prevTasksCache = null; private Set standbyTasksCache = null; + private Map taskLagsCache = null; static { // Just statically check to make sure that the generated code always stays in sync with the overall protocol @@ -69,12 +72,13 @@ private static void validateVersions(final int version, final int latestSupporte public SubscriptionInfo(final int version, final int latestSupportedVersion, final UUID processId, - final Set prevTasks, - final Set standbyTasks, - final String userEndPoint) { + final String userEndPoint, + final Map taskLags) { validateVersions(version, latestSupportedVersion); final SubscriptionInfoData data = new SubscriptionInfoData(); data.setVersion(version); + data.setProcessId(processId); + if (version >= 2) { data.setUserEndPoint(userEndPoint == null ? new byte[0] @@ -83,7 +87,38 @@ public SubscriptionInfo(final int version, if (version >= 3) { data.setLatestSupportedVersion(latestSupportedVersion); } - data.setProcessId(processId); + if (version >= 7) { + setTaskLagDataFromTaskLagMap(data, taskLags); + } else { + setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags); + } + this.data = data; + } + + private static void setTaskLagDataFromTaskLagMap(final SubscriptionInfoData data, + final Map taskLags) { + data.setTaskLags(taskLags.entrySet().stream().map(t -> { + final SubscriptionInfoData.TaskLagPair taskLagPair = new SubscriptionInfoData.TaskLagPair(); + taskLagPair.setTopicGroupId(t.getKey().topicGroupId); + taskLagPair.setPartition(t.getKey().partition); + taskLagPair.setLag(t.getValue()); + return taskLagPair; + }).collect(Collectors.toList())); + } + + private static void setPrevAndStandbySetsFromParsedTaskLagMap(final SubscriptionInfoData data, + final Map taskLags) { + final Set prevTasks = new HashSet<>(); + final Set standbyTasks = new HashSet<>(); + + for (Map.Entry taskLagEntry : taskLags.entrySet()) { + if (taskLagEntry.getValue() == -1) { + prevTasks.add(taskLagEntry.getKey()); + } else { + standbyTasks.add(taskLagEntry.getKey()); + } + } + data.setPrevTasks(prevTasks.stream().map(t -> { final SubscriptionInfoData.TaskId taskId = new SubscriptionInfoData.TaskId(); taskId.setTopicGroupId(t.topicGroupId); @@ -96,8 +131,6 @@ public SubscriptionInfo(final int version, taskId.setPartition(t.partition); return taskId; }).collect(Collectors.toList())); - - this.data = data; } private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) { @@ -141,6 +174,19 @@ public Set standbyTasks() { return standbyTasksCache; } + public Map taskLags() { + if (taskLagsCache == null) { + taskLagsCache = Collections.unmodifiableMap( + data.taskLags() + .stream() + .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), + l -> l.lag())) + + ); + } + return taskLagsCache; + } + public String userEndPoint() { return data.userEndPoint() == null || data.userEndPoint().length == 0 ? null diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index 09e95feeb58f1..3da81f6530be7 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-6", + "validVersions": "1-7", "fields": [ { "name": "version", @@ -35,18 +35,23 @@ }, { "name": "prevTasks", - "versions": "1+", + "versions": "1-6", "type": "[]TaskId" }, { "name": "standbyTasks", - "versions": "1+", + "versions": "1-6", "type": "[]TaskId" }, { "name": "userEndPoint", "versions": "2+", "type": "bytes" + }, + { + "name": "taskLags", + "versions": "7+", + "type": "[]TaskLagPair" } ], "commonStructs": [ @@ -65,6 +70,27 @@ "type": "int32" } ] + }, + { + "name": "TaskLagPair", + "versions": "7+", + "fields": [ + { + "name": "topicGroupId", + "versions": "1+", + "type": "int32" + }, + { + "name": "partition", + "versions": "1+", + "type": "int32" + }, + { + "name": "lag", + "versions": "1+", + "type": "int32" + } + ] } ], "type": "data" From 8b45f78aee5bcb3fa49830e1256ed2121fdd28be Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 14 Feb 2020 16:19:18 -0800 Subject: [PATCH 02/24] fixing up tests --- .../assignment/SubscriptionInfo.java | 50 ++++++++++++- .../LegacySubscriptionInfoSerde.java | 41 ++++++++--- .../assignment/SubscriptionInfoTest.java | 70 ++++++++++--------- 3 files changed, 117 insertions(+), 44 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 64c2dbc08dd33..8cfd0490b895e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -95,6 +95,41 @@ public SubscriptionInfo(final int version, this.data = data; } + public SubscriptionInfo(final int version, + final int latestSupportedVersion, + final UUID processId, + final Set prevTasks, + final Set standbyTasks, + final String userEndPoint) { + validateVersions(version, latestSupportedVersion); + if (version >= 7) { + throw new IllegalStateException("Version 7 subscriptions should use task lag map instead of prev and standby sets"); + } + + final SubscriptionInfoData data = new SubscriptionInfoData(); + data.setVersion(version); + data.setProcessId(processId); + + if (version >= 2) { + data.setUserEndPoint(userEndPoint == null + ? new byte[0] + : userEndPoint.getBytes(StandardCharsets.UTF_8)); + } + if (version >= 3) { + data.setLatestSupportedVersion(latestSupportedVersion); + } + + data.setPrevTasks(prevTasks); + data.setStandbyTasks(standbyTasks) + if (version >= 7) { + setTaskLagDataFromTaskLagMap(data, taskLags); + } else { + setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags); + } + this.data = data; + } + + private static void setTaskLagDataFromTaskLagMap(final SubscriptionInfoData data, final Map taskLags) { data.setTaskLags(taskLags.entrySet().stream().map(t -> { @@ -111,7 +146,7 @@ private static void setPrevAndStandbySetsFromParsedTaskLagMap(final Subscription final Set prevTasks = new HashSet<>(); final Set standbyTasks = new HashSet<>(); - for (Map.Entry taskLagEntry : taskLags.entrySet()) { + for (final Map.Entry taskLagEntry : taskLags.entrySet()) { if (taskLagEntry.getValue() == -1) { prevTasks.add(taskLagEntry.getKey()); } else { @@ -152,6 +187,11 @@ public UUID processId() { public Set prevTasks() { if (prevTasksCache == null) { + // lazily initialize the prev and standby task maps as they may not be needed + if (data.version() >= 7) { + setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); + + } prevTasksCache = Collections.unmodifiableSet( data.prevTasks() .stream() @@ -164,6 +204,11 @@ public Set prevTasks() { public Set standbyTasks() { if (standbyTasksCache == null) { + // lazily initialize the prev and standby task maps as they may not be needed + if (data.version() >= 7) { + setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); + + } standbyTasksCache = Collections.unmodifiableSet( data.standbyTasks() .stream() @@ -179,8 +224,7 @@ public Map taskLags() { taskLagsCache = Collections.unmodifiableMap( data.taskLags() .stream() - .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), - l -> l.lag())) + .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), l -> l.lag())) ); } 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 dd12cb859aa2d..d8e4f084feb44 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 @@ -16,6 +16,10 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; import org.slf4j.Logger; @@ -27,6 +31,7 @@ import java.util.HashSet; import java.util.Set; import java.util.UUID; +import scala.Int; public class LegacySubscriptionInfoSerde { @@ -41,13 +46,15 @@ public class LegacySubscriptionInfoSerde { private final Set prevTasks; private final Set standbyTasks; private final String userEndPoint; + private final Map taskLags; public LegacySubscriptionInfoSerde(final int version, final int latestSupportedVersion, final UUID processId, final Set prevTasks, final Set standbyTasks, - final String userEndPoint) { + final String userEndPoint, + final Map taskLags) { if (latestSupportedVersion == UNKNOWN && (version < 1 || version > 2)) { throw new IllegalArgumentException( "Only versions 1 and 2 are expected to use an UNKNOWN (-1) latest supported version. " + @@ -61,10 +68,11 @@ public LegacySubscriptionInfoSerde(final int version, usedVersion = version; this.latestSupportedVersion = latestSupportedVersion; this.processId = processId; - this.prevTasks = prevTasks; - this.standbyTasks = standbyTasks; + this.prevTasks = prevTasks == null ? taskLagMapToPrevTaskSet(taskLags) : prevTasks(); + this.standbyTasks = standbyTasks == null ? taskLagMapToStandbyTaskSet(taskLags) : standbyTasks(); // Coerce empty string to null. This was the effect of the serialization logic, anyway. this.userEndPoint = userEndPoint == null || userEndPoint.isEmpty() ? null : userEndPoint; + this.taskLags = taskLags == null ? prevAndStandbyTasksToTaskLagMap(prevTasks, standbyTasks) : taskLags; } public int version() { @@ -195,28 +203,28 @@ public static LegacySubscriptionInfoSerde decode(final ByteBuffer data) { data.rewind(); final int usedVersion = data.getInt(); - if (usedVersion == 4 || usedVersion == 3) { + if (usedVersion > 2 && usedVersion < 7) { final int latestSupportedVersion = data.getInt(); final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); final String userEndPoint = decodeUserEndpoint(data); - return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, processId, prevTasks, standbyTasks, userEndPoint); + return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, processId, prevTasks, standbyTasks, userEndPoint, null); } else if (usedVersion == 2) { final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); final String userEndPoint = decodeUserEndpoint(data); - return new LegacySubscriptionInfoSerde(2, UNKNOWN, processId, prevTasks, standbyTasks, userEndPoint); + return new LegacySubscriptionInfoSerde(2, UNKNOWN, processId, prevTasks, standbyTasks, userEndPoint, null); } else if (usedVersion == 1) { final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); - return new LegacySubscriptionInfoSerde(1, UNKNOWN, processId, prevTasks, standbyTasks, null); + return new LegacySubscriptionInfoSerde(1, UNKNOWN, processId, prevTasks, standbyTasks, null, null); } else { final int latestSupportedVersion = data.getInt(); log.info("Unable to decode subscription data: used version: {}; latest supported version: {}", usedVersion, LATEST_SUPPORTED_VERSION); - return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, null, null, null, null); + return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, null, null, null, null, null); } } @@ -236,6 +244,23 @@ private static Set decodeTasks(final ByteBuffer data) { return prevTasks; } + private static Set taskLagMapToPrevTaskSet(final Map taskLags) { + return taskLags.entrySet().stream().filter(t -> t.getValue() == 0).map(Map.Entry::getKey).collect(Collectors.toSet()); + } + + private static Set taskLagMapToStandbyTaskSet(final Map taskLags) { + return taskLags.entrySet().stream().filter(t -> t.getValue() != 0).map(Map.Entry::getKey).collect(Collectors.toSet()); + } + + private static Map prevAndStandbyTasksToTaskLagMap(final Set prevTasks, + final Set standbyTasks) { + final Map taskLags = new HashMap<>(prevTasks.stream() + .collect(Collectors.toMap(t -> t, l -> -1))); + taskLags.putAll(standbyTasks.stream() + .collect(Collectors.toMap(t -> t, l -> 0))); + return taskLags; + } + private static UUID decodeProcessId(final ByteBuffer data) { return new UUID(data.getLong(), data.getLong()); } 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 218ecbbb798b9..ad585f97b7537 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 @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.HashMap; +import java.util.Map; import org.apache.kafka.streams.processor.TaskId; import org.junit.Test; @@ -31,13 +33,21 @@ public class SubscriptionInfoTest { private final UUID processId = UUID.randomUUID(); - private final Set activeTasks = new HashSet<>(Arrays.asList( + private static final Set activeTasks = new HashSet<>(Arrays.asList( new TaskId(0, 0), new TaskId(0, 1), new TaskId(1, 0))); - private final Set standbyTasks = new HashSet<>(Arrays.asList( + private static final Set standbyTasks = new HashSet<>(Arrays.asList( new TaskId(1, 1), new TaskId(2, 0))); + private static final Map taskLags = new HashMap<>(); + static { + taskLags.put(new TaskId(0, 0), -1); + taskLags.put(new TaskId(0, 1), -1); + taskLags.put(new TaskId(1, 0), -1); + taskLags.put(new TaskId(1, 1), 0); + taskLags.put(new TaskId(2, 0), 10); + } private final static String IGNORED_USER_ENDPOINT = "ignoredUserEndpoint:80"; @@ -47,9 +57,8 @@ public void shouldThrowForUnknownVersion1() { 0, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); } @@ -59,9 +68,8 @@ public void shouldThrowForUnknownVersion2() { LATEST_SUPPORTED_VERSION + 1, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); } @@ -71,9 +79,8 @@ public void shouldEncodeAndDecodeVersion1() { 1, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - IGNORED_USER_ENDPOINT + IGNORED_USER_ENDPOINT, + taskLags ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(1, decoded.version()); @@ -90,9 +97,8 @@ public void generatedVersion1ShouldBeDecodableByLegacyLogic() { 1, 1234, processId, - activeTasks, - standbyTasks, - "ignoreme" + "ignoreme", + taskLags ); final ByteBuffer buffer = info.encode(); @@ -105,7 +111,6 @@ public void generatedVersion1ShouldBeDecodableByLegacyLogic() { assertNull(decoded.userEndPoint()); } - @Test public void generatedVersion1ShouldDecodeLegacyFormat() { final LegacySubscriptionInfoSerde info = new LegacySubscriptionInfoSerde( @@ -114,7 +119,8 @@ public void generatedVersion1ShouldDecodeLegacyFormat() { processId, activeTasks, standbyTasks, - "localhost:80" + "localhost:80", + null ); final ByteBuffer buffer = info.encode(); buffer.rewind(); @@ -133,9 +139,8 @@ public void shouldEncodeAndDecodeVersion2() { 2, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(2, decoded.version()); @@ -152,9 +157,8 @@ public void generatedVersion2ShouldBeDecodableByLegacyLogic() { 2, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); final ByteBuffer buffer = info.encode(); @@ -175,7 +179,8 @@ public void generatedVersion2ShouldDecodeLegacyFormat() { processId, activeTasks, standbyTasks, - "localhost:80" + "localhost:80", + null ); final ByteBuffer buffer = info.encode(); buffer.rewind(); @@ -195,9 +200,8 @@ public void shouldEncodeAndDecodeVersion3And4() { version, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(version, decoded.version()); @@ -216,9 +220,8 @@ public void generatedVersion3And4ShouldBeDecodableByLegacyLogic() { version, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, - "localhost:80" + "localhost:80", + taskLags ); final ByteBuffer buffer = info.encode(); @@ -241,7 +244,8 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { processId, activeTasks, standbyTasks, - "localhost:80" + "localhost:80", + null ); final ByteBuffer buffer = info.encode(); buffer.rewind(); @@ -258,7 +262,7 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { @Test public void shouldEncodeAndDecodeVersion5() { - final SubscriptionInfo info = new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, activeTasks, standbyTasks, "localhost:80"); + final SubscriptionInfo info = new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", taskLags); assertEquals(info, SubscriptionInfo.decode(info.encode())); } @@ -274,8 +278,8 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; final int latestSupportedVersion = LATEST_SUPPORTED_VERSION - 1; - final SubscriptionInfo info = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, activeTasks, standbyTasks, "localhost:80"); - final SubscriptionInfo expectedInfo = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, activeTasks, standbyTasks, "localhost:80"); + final SubscriptionInfo info = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", taskLags); + final SubscriptionInfo expectedInfo = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80"); assertEquals(expectedInfo, SubscriptionInfo.decode(info.encode())); } From b4d91a85a7eabbfdf2e0ebfa59d3e411302e7a88 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 14 Feb 2020 17:34:46 -0800 Subject: [PATCH 03/24] cleaned up task manager and subscription code, compiles --- .../internals/StreamsPartitionAssignor.java | 46 +-------- .../processor/internals/TaskManager.java | 23 ++++- .../assignment/SubscriptionInfo.java | 35 ------- .../StreamsPartitionAssignorTest.java | 17 +++- .../LegacySubscriptionInfoSerde.java | 15 ++- .../assignment/SubscriptionInfoTest.java | 99 +++++++++++-------- .../streams/tests/StreamsUpgradeTest.java | 41 +++----- 7 files changed, 118 insertions(+), 158 deletions(-) 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 da4795767c2eb..dbd1bf4d71e26 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 @@ -217,56 +217,19 @@ public List supportedProtocols() { public ByteBuffer subscriptionUserData(final Set topics) { // Adds the following information to subscription // 1. Client UUID (a unique id assigned to an instance of KafkaStreams) - // 2. Task ids of previously running tasks - // 3. Task ids of valid local states on the client's state directory. - final Set standbyTasks = taskManager.tasksOnLocalStorage(); - final Set activeTasks = prepareForSubscription(taskManager, - topics, - standbyTasks, - rebalanceProtocol); - - final Map taskLags = new HashMap<>(activeTasks.stream() - .collect(Collectors.toMap(t -> t, l -> -1))); - taskLags.putAll(standbyTasks.stream() - .collect(Collectors.toMap(t -> t, l -> 0))); + // 2. Map from task id to its overall lag + + taskManager.handleRebalanceStart(topics); return new SubscriptionInfo( usedSubscriptionMetadataVersion, LATEST_SUPPORTED_VERSION, taskManager.processId(), userEndPoint, - taskLags) + taskManager.getTaskLags()) .encode(); } - protected static Set prepareForSubscription(final TaskManager taskManager, - final Set topics, - final Set standbyTasks, - final RebalanceProtocol rebalanceProtocol) { - // Any tasks that are not yet running are counted as standby tasks for assignment purposes, - // along with any old tasks for which we still found state on disk - final Set activeTasks; - - switch (rebalanceProtocol) { - case EAGER: - // In eager, onPartitionsRevoked is called first and we must get the previously saved running task ids - activeTasks = taskManager.activeTaskIds(); - standbyTasks.removeAll(activeTasks); - break; - case COOPERATIVE: - // In cooperative, we will use the encoded ownedPartitions to determine the running tasks - activeTasks = Collections.emptySet(); - standbyTasks.removeAll(taskManager.activeTaskIds()); - break; - default: - throw new IllegalStateException("Streams partition assignor's rebalance protocol is unknown"); - } - - taskManager.handleRebalanceStart(topics); - - return activeTasks; - } - private Map errorAssignment(final Map clientsMetadata, final String topic, final int errorCode) { @@ -1301,4 +1264,5 @@ protected String userEndPoint() { protected TaskManager taskManger() { return taskManager; } + } 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 dc14d31f71285..b46daffb34f89 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 @@ -53,6 +53,8 @@ import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; public class TaskManager { + static final int ACTIVE_TASK_SENTINEL_LAG = -1; + // initialize the task list // activeTasks needs to be concurrent as it can be accessed // by QueryableState @@ -116,7 +118,8 @@ boolean isRebalanceInProgress() { return rebalanceInProgress; } - void handleRebalanceStart(final Set subscribedTopics) { + // visible for testing + public void handleRebalanceStart(final Set subscribedTopics) { builder.addSubscribedTopicsFromMetadata(subscribedTopics, logPrefix); rebalanceInProgress = true; @@ -354,11 +357,27 @@ void handleLostAll() { } } + /** + * @return Map from task id to that task's overall lag across all state stores + */ + public Map getTaskLags() { + final Map taskLags = new HashMap<>(); + + for (final TaskId id : tasksOnLocalStorage()) { + if (activeTaskMap().containsKey(id)) { + taskLags.put(id, ACTIVE_TASK_SENTINEL_LAG); + } else { + taskLags.put(id, 0); + } + } + return taskLags; + } + /** * Returns ids of tasks whose states are kept on the local storage. This includes active, standby, and previously * assigned but not yet cleaned up tasks */ - public Set tasksOnLocalStorage() { + Set tasksOnLocalStorage() { // A client could contain some inactive tasks whose states are still kept on the local storage in the following scenarios: // 1) the client is actively maintaining standby tasks by maintaining their states from the change log. // 2) the client has just got some tasks migrated out of itself to other clients while these task states diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 8cfd0490b895e..b88739686dc67 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -95,41 +95,6 @@ public SubscriptionInfo(final int version, this.data = data; } - public SubscriptionInfo(final int version, - final int latestSupportedVersion, - final UUID processId, - final Set prevTasks, - final Set standbyTasks, - final String userEndPoint) { - validateVersions(version, latestSupportedVersion); - if (version >= 7) { - throw new IllegalStateException("Version 7 subscriptions should use task lag map instead of prev and standby sets"); - } - - final SubscriptionInfoData data = new SubscriptionInfoData(); - data.setVersion(version); - data.setProcessId(processId); - - if (version >= 2) { - data.setUserEndPoint(userEndPoint == null - ? new byte[0] - : userEndPoint.getBytes(StandardCharsets.UTF_8)); - } - if (version >= 3) { - data.setLatestSupportedVersion(latestSupportedVersion); - } - - data.setPrevTasks(prevTasks); - data.setStandbyTasks(standbyTasks) - if (version >= 7) { - setTaskLagDataFromTaskLagMap(data, taskLags); - } else { - setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags); - } - this.data = data; - } - - private static void setTaskLagDataFromTaskLagMap(final SubscriptionInfoData data, final Map taskLags) { data.setTaskLags(taskLags.entrySet().stream().map(t -> { 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 f00f93332e2a6..6c404e462174f 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 @@ -204,8 +204,8 @@ private void createMockTaskManager(final Set prevTasks, final InternalTopologyBuilder builder) { taskManager = EasyMock.createNiceMock(TaskManager.class); EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes(); - EasyMock.expect(taskManager.activeTaskIds()).andReturn(prevTasks).anyTimes(); - EasyMock.expect(taskManager.tasksOnLocalStorage()).andReturn(cachedTasks).anyTimes(); + //EasyMock.expect(taskManager.activeTaskIds()).andReturn(prevTasks).anyTimes(); + EasyMock.expect(taskManager.getTaskLags()).andReturn(getTaskLags(prevTasks, cachedTasks)).anyTimes(); EasyMock.expect(taskManager.processId()).andReturn(processId).anyTimes(); builder.setApplicationId(APPLICATION_ID); builder.buildTopology(); @@ -227,18 +227,21 @@ private static SubscriptionInfo getInfo(final int version, final Set prevTasks, final Set standbyTasks, final String userEndPoint) { - return new SubscriptionInfo(version, LATEST_SUPPORTED_VERSION, processId, prevTasks, standbyTasks, userEndPoint); + return new SubscriptionInfo( + version, LATEST_SUPPORTED_VERSION, processId, userEndPoint, getTaskLags(prevTasks, standbyTasks)); } private static SubscriptionInfo getInfo(final UUID processId, final Set prevTasks, final Set standbyTasks, final String userEndPoint) { - return new SubscriptionInfo(LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, prevTasks, standbyTasks, userEndPoint); + return new SubscriptionInfo( + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, userEndPoint, getTaskLags(prevTasks, standbyTasks)); } private static SubscriptionInfo getInfo(final UUID processId, final Set prevTasks, final Set standbyTasks) { - return new SubscriptionInfo(LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, prevTasks, standbyTasks, USER_END_POINT); + return new SubscriptionInfo( + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, USER_END_POINT, getTaskLags(prevTasks, standbyTasks)); } @Test @@ -1993,4 +1996,8 @@ private static void assertEquivalentAssignment(final Map> t } } + static Map getTaskLags(final Set activeTasks, final Set cachedTasks) { + return cachedTasks.stream().collect(Collectors.toMap(t -> t, l -> activeTasks.contains(l) ? -1 : 0)); + } + } 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 d8e4f084feb44..e91f89221f7b1 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import java.util.Collections; import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; @@ -31,7 +30,6 @@ import java.util.HashSet; import java.util.Set; import java.util.UUID; -import scala.Int; public class LegacySubscriptionInfoSerde { @@ -179,13 +177,22 @@ public static void encodeTasks(final ByteBuffer buf, } public static void encodeUserEndPoint(final ByteBuffer buf, - final byte[] endPointBytes) { + final byte[] endPointBytes) { if (endPointBytes != null) { buf.putInt(endPointBytes.length); buf.put(endPointBytes); } } + public static void encodeTaskLags(final ByteBuffer buf, + final Map taskLags) { + buf.putInt(taskLags.size()); + for (final Map.Entry taskLagEntry : taskLags.entrySet()) { + taskLagEntry.getKey().writeTo(buf); + buf.putInt(taskLagEntry.getValue()); + } + } + public static byte[] prepareUserEndPoint(final String userEndPoint) { if (userEndPoint == null) { return new byte[0]; @@ -203,7 +210,7 @@ public static LegacySubscriptionInfoSerde decode(final ByteBuffer data) { data.rewind(); final int usedVersion = data.getInt(); - if (usedVersion > 2 && usedVersion < 7) { + if (usedVersion > 2 && usedVersion < 7) { final int latestSupportedVersion = data.getInt(); final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); 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 ad585f97b7537..608e7336d5d3a 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 @@ -33,20 +33,20 @@ public class SubscriptionInfoTest { private final UUID processId = UUID.randomUUID(); - private static final Set activeTasks = new HashSet<>(Arrays.asList( + private static final Set ACTIVE_TASKS = new HashSet<>(Arrays.asList( new TaskId(0, 0), new TaskId(0, 1), new TaskId(1, 0))); - private static final Set standbyTasks = new HashSet<>(Arrays.asList( + private static final Set STANDBY_TASKS = new HashSet<>(Arrays.asList( new TaskId(1, 1), new TaskId(2, 0))); - private static final Map taskLags = new HashMap<>(); + private static final Map TASK_LAGS = new HashMap<>(); static { - taskLags.put(new TaskId(0, 0), -1); - taskLags.put(new TaskId(0, 1), -1); - taskLags.put(new TaskId(1, 0), -1); - taskLags.put(new TaskId(1, 1), 0); - taskLags.put(new TaskId(2, 0), 10); + TASK_LAGS.put(new TaskId(0, 0), -1); + TASK_LAGS.put(new TaskId(0, 1), -1); + TASK_LAGS.put(new TaskId(1, 0), -1); + TASK_LAGS.put(new TaskId(1, 1), 0); + TASK_LAGS.put(new TaskId(2, 0), 10); } private final static String IGNORED_USER_ENDPOINT = "ignoredUserEndpoint:80"; @@ -58,7 +58,7 @@ public void shouldThrowForUnknownVersion1() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); } @@ -69,7 +69,7 @@ public void shouldThrowForUnknownVersion2() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); } @@ -80,14 +80,14 @@ public void shouldEncodeAndDecodeVersion1() { LATEST_SUPPORTED_VERSION, processId, IGNORED_USER_ENDPOINT, - taskLags + TASK_LAGS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(1, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertNull(decoded.userEndPoint()); } @@ -98,7 +98,7 @@ public void generatedVersion1ShouldBeDecodableByLegacyLogic() { 1234, processId, "ignoreme", - taskLags + TASK_LAGS ); final ByteBuffer buffer = info.encode(); @@ -106,8 +106,8 @@ public void generatedVersion1ShouldBeDecodableByLegacyLogic() { assertEquals(1, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertNull(decoded.userEndPoint()); } @@ -117,8 +117,8 @@ public void generatedVersion1ShouldDecodeLegacyFormat() { 1, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, + ACTIVE_TASKS, + STANDBY_TASKS, "localhost:80", null ); @@ -128,8 +128,8 @@ public void generatedVersion1ShouldDecodeLegacyFormat() { assertEquals(1, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertNull(decoded.userEndPoint()); } @@ -140,14 +140,14 @@ public void shouldEncodeAndDecodeVersion2() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(2, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertEquals("localhost:80", decoded.userEndPoint()); } @@ -158,7 +158,7 @@ public void generatedVersion2ShouldBeDecodableByLegacyLogic() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); final ByteBuffer buffer = info.encode(); @@ -166,8 +166,8 @@ public void generatedVersion2ShouldBeDecodableByLegacyLogic() { assertEquals(2, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertEquals("localhost:80", decoded.userEndPoint()); } @@ -177,8 +177,8 @@ public void generatedVersion2ShouldDecodeLegacyFormat() { 2, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, + ACTIVE_TASKS, + STANDBY_TASKS, "localhost:80", null ); @@ -188,8 +188,8 @@ public void generatedVersion2ShouldDecodeLegacyFormat() { assertEquals(2, decoded.version()); assertEquals(SubscriptionInfo.UNKNOWN, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertEquals("localhost:80", decoded.userEndPoint()); } @@ -201,14 +201,14 @@ public void shouldEncodeAndDecodeVersion3And4() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(version, decoded.version()); assertEquals(LATEST_SUPPORTED_VERSION, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertEquals("localhost:80", decoded.userEndPoint()); } } @@ -221,7 +221,7 @@ public void generatedVersion3And4ShouldBeDecodableByLegacyLogic() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - taskLags + TASK_LAGS ); final ByteBuffer buffer = info.encode(); @@ -229,8 +229,8 @@ public void generatedVersion3And4ShouldBeDecodableByLegacyLogic() { assertEquals(version, decoded.version()); assertEquals(LATEST_SUPPORTED_VERSION, decoded.latestSupportedVersion()); assertEquals(processId, decoded.processId()); - assertEquals(activeTasks, decoded.prevTasks()); - assertEquals(standbyTasks, decoded.standbyTasks()); + assertEquals(ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(STANDBY_TASKS, decoded.standbyTasks()); assertEquals("localhost:80", decoded.userEndPoint()); } } @@ -242,8 +242,8 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { version, LATEST_SUPPORTED_VERSION, processId, - activeTasks, - standbyTasks, + ACTIVE_TASKS, + STANDBY_TASKS, "localhost:80", null ); @@ -254,15 +254,16 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { assertEquals(message, version, decoded.version()); assertEquals(message, LATEST_SUPPORTED_VERSION, decoded.latestSupportedVersion()); assertEquals(message, processId, decoded.processId()); - assertEquals(message, activeTasks, decoded.prevTasks()); - assertEquals(message, standbyTasks, decoded.standbyTasks()); + assertEquals(message, ACTIVE_TASKS, decoded.prevTasks()); + assertEquals(message, STANDBY_TASKS, decoded.standbyTasks()); assertEquals(message, "localhost:80", decoded.userEndPoint()); } } @Test public void shouldEncodeAndDecodeVersion5() { - final SubscriptionInfo info = new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", taskLags); + final SubscriptionInfo info = new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", + TASK_LAGS); assertEquals(info, SubscriptionInfo.decode(info.encode())); } @@ -278,11 +279,23 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; final int latestSupportedVersion = LATEST_SUPPORTED_VERSION - 1; - final SubscriptionInfo info = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", taskLags); - final SubscriptionInfo expectedInfo = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80"); + final SubscriptionInfo info = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", + TASK_LAGS); + final SubscriptionInfo expectedInfo = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", + TASK_LAGS); assertEquals(expectedInfo, SubscriptionInfo.decode(info.encode())); } + @Test + public void shouldEncodeVersion7() { + //TODO-soph + } + + @Test + public void shouldConvertTaskLagMapToTaskSetsForOlderVersion() { + // TODO-soph + } + private static ByteBuffer encodeFutureVersion() { final ByteBuffer buf = ByteBuffer.allocate(4 /* used version */ + 4 /* supported version */); 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 c979846a56ec4..c345211ab628e 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 @@ -146,30 +146,22 @@ public ByteBuffer subscriptionUserData(final Set topics) { // 2. Task ids of previously running tasks // 3. Task ids of valid local states on the client's state directory. final TaskManager taskManager = taskManger(); - - final Set standbyTasks = taskManager.tasksOnLocalStorage(); - final Set activeTasks = prepareForSubscription(taskManager, - topics, - standbyTasks, - REBALANCE_PROTOCOL); - + taskManager.handleRebalanceStart(topics); if (usedSubscriptionMetadataVersion <= LATEST_SUPPORTED_VERSION) { return new SubscriptionInfo( usedSubscriptionMetadataVersion, LATEST_SUPPORTED_VERSION + 1, taskManager.processId(), - activeTasks, - standbyTasks, - userEndPoint() + userEndPoint(), + taskManager.getTaskLags() ).encode(); } else { return new FutureSubscriptionInfo( usedSubscriptionMetadataVersion, taskManager.processId(), - activeTasks, - standbyTasks, - userEndPoint()) + userEndPoint(), + taskManager.getTaskLags()) .encode(); } } @@ -258,9 +250,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, info.processId(), - info.prevTasks(), - info.standbyTasks(), - info.userEndPoint()) + info.userEndPoint(), taskManger().getTaskLags()) .encode(), subscription.ownedPartitions() )); @@ -291,21 +281,18 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr private static class FutureSubscriptionInfo { private final int version; private final UUID processId; - private final Set prevTasks; - private final Set standbyTasks; private final String userEndPoint; + private final Map taskLags; // for testing only; don't apply version checks FutureSubscriptionInfo(final int version, final UUID processId, - final Set prevTasks, - final Set standbyTasks, - final String userEndPoint) { + final String userEndPoint, + final Map taskLags) { this.version = version; this.processId = processId; - this.prevTasks = prevTasks; - this.standbyTasks = standbyTasks; this.userEndPoint = userEndPoint; + this.taskLags = taskLags; if (version <= LATEST_SUPPORTED_VERSION) { throw new IllegalArgumentException("this class can't be used with version " + version); } @@ -324,17 +311,15 @@ private ByteBuffer encodeFutureVersion() { 4 + // used version 4 + // latest supported version version 16 + // client ID - 4 + prevTasks.size() * 8 + // length + prev tasks - 4 + standbyTasks.size() * 8 + // length + standby tasks - 4 + endPointBytes.length + 4 + endPointBytes.length + // length + endpoint + 4 + taskLags.size() * 12 // length + task lag info ); buf.putInt(version); // used version buf.putInt(version); // supported version LegacySubscriptionInfoSerde.encodeClientUUID(buf, processId); - LegacySubscriptionInfoSerde.encodeTasks(buf, prevTasks); - LegacySubscriptionInfoSerde.encodeTasks(buf, standbyTasks); LegacySubscriptionInfoSerde.encodeUserEndPoint(buf, endPointBytes); + LegacySubscriptionInfoSerde.encodeTaskLags(buf, taskLags); buf.rewind(); From 2aabf29def59f348ce2abdb60771d8d60878e220 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 14 Feb 2020 17:56:56 -0800 Subject: [PATCH 04/24] bump assignment version handling --- .../internals/StreamsPartitionAssignor.java | 1 + .../internals/assignment/AssignmentInfo.java | 2 ++ .../internals/assignment/SubscriptionInfoTest.java | 12 ++++++------ 3 files changed, 9 insertions(+), 6 deletions(-) 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 dbd1bf4d71e26..9708ff944af5e 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 @@ -1122,6 +1122,7 @@ public void onAssignment(final Assignment assignment, final ConsumerGroupMetadat topicToPartitionInfo = getTopicPartitionInfo(partitionsByHost); break; case 6: + case 7: validateActiveTaskEncoding(partitions, info, logPrefix); activeTasks = getActiveTasks(partitions, info); 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 f7d8541c0326b..d9501be68c203 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 @@ -163,6 +163,7 @@ public ByteBuffer encode() { out.writeInt(errCode); break; case 6: + case 7: out.writeInt(usedVersion); out.writeInt(commonlySupportedVersion); encodeActiveAndStandbyTaskAssignment(out); @@ -327,6 +328,7 @@ public static AssignmentInfo decode(final ByteBuffer data) { assignmentInfo.errCode = in.readInt(); break; case 6: + case 7: commonlySupportedVersion = in.readInt(); assignmentInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion); decodeActiveTasks(assignmentInfo, in); 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 608e7336d5d3a..f854435359790 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 @@ -262,8 +262,8 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { @Test public void shouldEncodeAndDecodeVersion5() { - final SubscriptionInfo info = new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS); + final SubscriptionInfo info = + new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); assertEquals(info, SubscriptionInfo.decode(info.encode())); } @@ -279,10 +279,10 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { final int usedVersion = LATEST_SUPPORTED_VERSION - 1; final int latestSupportedVersion = LATEST_SUPPORTED_VERSION - 1; - final SubscriptionInfo info = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", - TASK_LAGS); - final SubscriptionInfo expectedInfo = new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", - TASK_LAGS); + final SubscriptionInfo info = + new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_LAGS); + final SubscriptionInfo expectedInfo = + new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_LAGS); assertEquals(expectedInfo, SubscriptionInfo.decode(info.encode())); } From f510057179581229975f58666352c20077571f68 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 14 Feb 2020 18:00:47 -0800 Subject: [PATCH 05/24] add subscription info tests --- .../internals/assignment/SubscriptionInfoTest.java | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) 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 f854435359790..b9012e0be17ee 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 @@ -287,13 +287,17 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { } @Test - public void shouldEncodeVersion7() { - //TODO-soph + public void shouldEncodeAndDecodeVersion7() { + final SubscriptionInfo info = + new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); + assertEquals(info, SubscriptionInfo.decode(info.encode())); } @Test public void shouldConvertTaskLagMapToTaskSetsForOlderVersion() { - // TODO-soph + final SubscriptionInfo info = new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); + assertEquals(info.prevTasks(), ACTIVE_TASKS); + assertEquals(info.standbyTasks(), STANDBY_TASKS); } private static ByteBuffer encodeFutureVersion() { From c063dc42a6412815367fbb0dd6557c2e20cae05b Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Tue, 18 Feb 2020 18:02:04 -0800 Subject: [PATCH 06/24] fix assignmentinfo test --- .../processor/internals/assignment/AssignmentInfoTest.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 1b5fe749cfa73..4958503e57258 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 @@ -130,8 +130,8 @@ public void shouldEncodeAndDecodeVersion6() { @Test public void shouldEncodeAndDecodeSmallerCommonlySupportedVersion() { - final int usedVersion = LATEST_SUPPORTED_VERSION - 1; - final int commonlySupportedVersion = LATEST_SUPPORTED_VERSION - 1; + final int usedVersion = 5; + final int commonlySupportedVersion = 5; final AssignmentInfo info = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, activeAssignment, standbyAssignment, 2); final AssignmentInfo expectedInfo = new AssignmentInfo(usedVersion, commonlySupportedVersion, activeTasks, standbyTasks, From 424af9ae596cd2bb74869a577168a55f0254db72 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Tue, 18 Feb 2020 20:43:58 -0800 Subject: [PATCH 07/24] fixing up tests --- .../internals/StreamsPartitionAssignor.java | 15 +- .../processor/internals/TaskManager.java | 6 +- .../assignment/SubscriptionInfo.java | 3 - .../StreamsPartitionAssignorTest.java | 216 +++++++++--------- .../LegacySubscriptionInfoSerde.java | 7 +- 5 files changed, 120 insertions(+), 127 deletions(-) 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 9708ff944af5e..590429350df59 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 @@ -282,7 +282,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // keep track of any future consumers in a "dummy" Client since we can't decipher their subscription final UUID futureId = randomUUID(); final ClientMetadata futureClient = new ClientMetadata(null); - clientMetadataMap.put(futureId, futureClient); int minReceivedMetadataVersion = LATEST_SUPPORTED_VERSION; int minSupportedMetadataVersion = LATEST_SUPPORTED_VERSION; @@ -301,6 +300,9 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr if (usedVersion > LATEST_SUPPORTED_VERSION) { futureMetadataVersion = usedVersion; processId = futureId; + if (!clientMetadataMap.containsKey(futureId)) { + clientMetadataMap.put(futureId, futureClient); + } } else { processId = info.processId(); } @@ -313,7 +315,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr clientMetadataMap.put(info.processId(), clientMetadata); } - // add the consumer and any info its its subscription to the client + // add the consumer and any info in its subscription to the client clientMetadata.addConsumer(consumerId, subscription.ownedPartitions()); allOwnedPartitions.addAll(subscription.ownedPartitions()); clientMetadata.addPreviousTasks(info); @@ -557,12 +559,13 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final Map states = new HashMap<>(); for (final Map.Entry entry : clientMetadataMap.entrySet()) { + final UUID uuid = entry.getKey(); final ClientState state = entry.getValue().state; - states.put(entry.getKey(), state); + states.put(uuid, state); - // Either the active tasks (eager) OR the owned partitions (cooperative) were encoded in the subscription - // according to the rebalancing protocol, so convert any partitions in a client to tasks where necessary - if (!state.ownedPartitions().isEmpty()) { + // this is an optimization: we can't decode the future subscription's prev tasks, but we can figure them + // out from the encoded ownedPartitions + if (uuid == futureId && !state.ownedPartitions().isEmpty()) { final Set previousActiveTasks = new HashSet<>(); for (final Map.Entry partitionEntry : state.ownedPartitions().entrySet()) { final TopicPartition tp = partitionEntry.getKey(); 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 b46daffb34f89..4c785f331b177 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 @@ -364,7 +364,7 @@ public Map getTaskLags() { final Map taskLags = new HashMap<>(); for (final TaskId id : tasksOnLocalStorage()) { - if (activeTaskMap().containsKey(id)) { + if (isActive(id)) { taskLags.put(id, ACTIVE_TASK_SENTINEL_LAG); } else { taskLags.put(id, 0); @@ -491,6 +491,10 @@ private Stream standbyTaskStream() { return tasks.values().stream().filter(t -> !t.isActive()); } + private boolean isActive(final TaskId task) { + return tasks.get(task).isActive(); + } + /** * @throws TaskMigratedException if committing offsets failed (non-EOS) * or if the task producer got fenced (EOS) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index b88739686dc67..02fc64f6fbf82 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -155,7 +155,6 @@ public Set prevTasks() { // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); - } prevTasksCache = Collections.unmodifiableSet( data.prevTasks() @@ -172,7 +171,6 @@ public Set standbyTasks() { // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); - } standbyTasksCache = Collections.unmodifiableSet( data.standbyTasks() @@ -190,7 +188,6 @@ public Map taskLags() { data.taskLags() .stream() .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), l -> l.lag())) - ); } return taskLagsCache; 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 6c404e462174f..1a76662780200 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,9 +17,10 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.clients.admin.Admin; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Assignment; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.GroupSubscription; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; +import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.Subscription; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Node; @@ -199,19 +200,18 @@ private void createMockTaskManager() { } private void createMockTaskManager(final Set prevTasks, - final Set cachedTasks, + final Set standbyTasks, final UUID processId, final InternalTopologyBuilder builder) { taskManager = EasyMock.createNiceMock(TaskManager.class); EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes(); - //EasyMock.expect(taskManager.activeTaskIds()).andReturn(prevTasks).anyTimes(); - EasyMock.expect(taskManager.getTaskLags()).andReturn(getTaskLags(prevTasks, cachedTasks)).anyTimes(); + EasyMock.expect(taskManager.getTaskLags()).andReturn(getTaskLags(prevTasks, standbyTasks)).anyTimes(); EasyMock.expect(taskManager.processId()).andReturn(processId).anyTimes(); builder.setApplicationId(APPLICATION_ID); builder.buildTopology(); } - private Map subscriptions; + private Map subscriptions; @Before public void setUp() { @@ -232,11 +232,10 @@ private static SubscriptionInfo getInfo(final int version, } private static SubscriptionInfo getInfo(final UUID processId, - final Set prevTasks, - final Set standbyTasks, + final Map taskLags, final String userEndPoint) { return new SubscriptionInfo( - LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, userEndPoint, getTaskLags(prevTasks, standbyTasks)); + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, userEndPoint, taskLags); } private static SubscriptionInfo getInfo(final UUID processId, final Set prevTasks, final Set standbyTasks) { @@ -434,13 +433,12 @@ public void testEagerSubscription() { final Set prevTasks = mkSet( new TaskId(0, 1), new TaskId(1, 1), new TaskId(2, 1) ); - final Set cachedTasks = mkSet( - new TaskId(0, 1), new TaskId(1, 1), new TaskId(2, 1), + final Set standbyTasks = mkSet( new TaskId(0, 2), new TaskId(1, 2), new TaskId(2, 2) ); final UUID processId = UUID.randomUUID(); - createMockTaskManager(prevTasks, cachedTasks, processId, builder); + createMockTaskManager(prevTasks, standbyTasks, processId, builder); EasyMock.replay(taskManager); streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class); @@ -448,17 +446,13 @@ public void testEagerSubscription() { partitionAssignor.setRebalanceProtocol(RebalanceProtocol.EAGER); final Set topics = mkSet("topic1", "topic2"); - final ConsumerPartitionAssignor.Subscription subscription = new ConsumerPartitionAssignor.Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); + final Subscription subscription = new Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); Collections.sort(subscription.topics()); assertEquals(asList("topic1", "topic2"), subscription.topics()); - final Set standbyTasks = new HashSet<>(cachedTasks); - standbyTasks.removeAll(prevTasks); - - // When following the eager protocol, we must encode the previous tasks ourselves since we must revoke - // everything and thus the "ownedPartitions" field in the subscription will be empty - final SubscriptionInfo info = getInfo(processId, prevTasks, standbyTasks, null); + + final SubscriptionInfo info = getInfo(processId, getTaskLags(prevTasks, standbyTasks), null); assertEquals(info, SubscriptionInfo.decode(subscription.userData())); } @@ -482,7 +476,7 @@ public void testCooperativeSubscription() { configurePartitionAssignor(emptyMap()); final Set topics = mkSet("topic1", "topic2"); - final ConsumerPartitionAssignor.Subscription subscription = new ConsumerPartitionAssignor.Subscription( + final Subscription subscription = new Subscription( new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); Collections.sort(subscription.topics()); @@ -491,9 +485,7 @@ public void testCooperativeSubscription() { final Set standbyTasks = new HashSet<>(cachedTasks); standbyTasks.removeAll(prevTasks); - // We don't encode the active tasks when following the cooperative protocol, as these are inferred from the - // ownedPartitions encoded in the subscription - final SubscriptionInfo info = getInfo(processId, Collections.emptySet(), standbyTasks, null); + final SubscriptionInfo info = getInfo(processId, getTaskLags(prevTasks, cachedTasks), null); assertEquals(info, SubscriptionInfo.decode(subscription.userData())); } @@ -523,22 +515,22 @@ public void testAssignBasic() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, prevTasks10, standbyTasks10, USER_END_POINT).encode() + getInfo(uuid1, getTaskLags(prevTasks10, standbyTasks10), USER_END_POINT).encode() )); subscriptions.put("consumer11", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, prevTasks11, standbyTasks11, USER_END_POINT).encode() + getInfo(uuid1, getTaskLags(prevTasks11, standbyTasks11), USER_END_POINT).encode() )); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid2, prevTasks20, standbyTasks20, USER_END_POINT).encode() + getInfo(uuid2, getTaskLags(prevTasks20, standbyTasks20), USER_END_POINT).encode() )); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assigned partitions assertEquals(mkSet(mkSet(t1p0, t2p0), mkSet(t1p1, t2p1)), @@ -616,17 +608,17 @@ public void shouldAssignEvenlyAcrossConsumersOneClientMultipleThreads() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, new HashSet<>(), new HashSet<>()).encode() )); subscriptions.put("consumer11", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, new HashSet<>(), new HashSet<>()).encode() )); - final Map assignments = partitionAssignor.assign(localMetadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignments = partitionAssignor.assign(localMetadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assigned partitions assertEquals(mkSet(mkSet(t2p2, t1p0, t1p2, t2p0), mkSet(t1p1, t2p1, t1p3, t2p3)), @@ -668,11 +660,11 @@ public void testAssignWithPartialTopology() { // will throw exception if it fails subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, emptyTasks, emptyTasks).encode() )); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assignment info final AssignmentInfo info10 = checkAssignment(mkSet("topic1"), assignments.get("consumer10")); @@ -705,13 +697,13 @@ public void testAssignEmptyMetadata() { configurePartitionAssignor(emptyMap()); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, prevTasks10, standbyTasks10).encode() )); // initially metadata is empty - Map assignments = + Map assignments = partitionAssignor.assign(emptyMetadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assigned partitions @@ -765,19 +757,19 @@ public void testAssignWithNewTasks() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, prevTasks10, emptyTasks).encode())); subscriptions.put("consumer11", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, prevTasks11, emptyTasks).encode())); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid2, prevTasks20, emptyTasks).encode())); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assigned partitions: since there is no previous task for topic 3 it will be assigned randomly so we cannot check exact match // also note that previously assigned partitions / tasks may not stay on the previous host since we may assign the new task first and @@ -832,16 +824,16 @@ public void testAssignWithStates() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription(topics, + new Subscription(topics, getInfo(uuid1, emptyTasks, emptyTasks).encode())); subscriptions.put("consumer11", - new ConsumerPartitionAssignor.Subscription(topics, + new Subscription(topics, getInfo(uuid1, emptyTasks, emptyTasks).encode())); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription(topics, + new Subscription(topics, getInfo(uuid2, emptyTasks, emptyTasks).encode())); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); // check assigned partition size: since there is no previous task and there are two sub-topologies the assignment is random so we cannot check exact match assertEquals(2, assignments.get("consumer10").partitions().size()); @@ -910,15 +902,15 @@ public void testAssignWithStandbyReplicasAndStatelessTasks() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, mkSet(task0_0), Collections.emptySet(), "any:9096").encode())); + getInfo(uuid1, getTaskLags(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid2, mkSet(task0_2), Collections.emptySet(), "any:9097").encode())); + getInfo(uuid2, getTaskLags(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); - final Map assignments = + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); final AssignmentInfo info10 = checkAssignment(allTopics, assignments.get("consumer10")); @@ -949,15 +941,15 @@ public void testAssignWithStandbyReplicasAndLoggingDisabled() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, mkSet(task0_0), Collections.emptySet(), "any:9096").encode())); + getInfo(uuid1, getTaskLags(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid2, mkSet(task0_2), Collections.emptySet(), "any:9097").encode())); + getInfo(uuid2, getTaskLags(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); - final Map assignments = + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); final AssignmentInfo info10 = checkAssignment(allTopics, assignments.get("consumer10")); @@ -1005,19 +997,19 @@ public void testAssignWithStandbyReplicas() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, prevTasks00, standbyTasks01, "any:9096").encode())); + getInfo(uuid1, getTaskLags(prevTasks00, standbyTasks01), "any:9096").encode())); subscriptions.put("consumer11", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid1, prevTasks01, standbyTasks02, "any:9096").encode())); + getInfo(uuid1, getTaskLags(prevTasks01, standbyTasks02), "any:9096").encode())); subscriptions.put("consumer20", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, - getInfo(uuid2, prevTasks02, standbyTasks00, "any:9097").encode())); + getInfo(uuid2, getTaskLags(prevTasks02, standbyTasks00), "any:9097").encode())); - final Map assignments = + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); // the first consumer @@ -1103,7 +1095,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, emptyMap(), 0); - final ConsumerPartitionAssignor.Assignment assignment = new ConsumerPartitionAssignor.Assignment(asList(t3p0, t3p3), info.encode()); + final Assignment assignment = new Assignment(asList(t3p0, t3p3), info.encode()); partitionAssignor.onAssignment(assignment, null); @@ -1135,7 +1127,7 @@ public void testAssignWithInternalTopics() { partitionAssignor.setInternalTopicManager(internalTopicManager); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, emptyTasks, emptyTasks).encode()) ); @@ -1171,7 +1163,7 @@ public void testAssignWithInternalTopicThatsSourceIsAnotherInternalTopic() { partitionAssignor.setInternalTopicManager(internalTopicManager); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, emptyTasks, emptyTasks).encode()) ); @@ -1222,11 +1214,11 @@ public void shouldGenerateTasksForAllCreatedPartitions() { partitionAssignor.setInternalTopicManager(mockInternalTopicManager); subscriptions.put(client, - new ConsumerPartitionAssignor.Subscription( + new Subscription( asList("topic1", "topic3"), getInfo(uuid, emptyTasks, emptyTasks).encode()) ); - final Map assignment = + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)) .groupAssignment(); @@ -1275,8 +1267,8 @@ public void shouldAddUserDefinedEndPointToSubscription() { configurePartitionAssignor(Collections.singletonMap(StreamsConfig.APPLICATION_SERVER_CONFIG, USER_END_POINT)); final Set topics = mkSet("input"); final ByteBuffer userData = partitionAssignor.subscriptionUserData(topics); - final ConsumerPartitionAssignor.Subscription subscription = - new ConsumerPartitionAssignor.Subscription(new ArrayList<>(topics), userData); + final Subscription subscription = + new Subscription(new ArrayList<>(topics), userData); final SubscriptionInfo subscriptionInfo = SubscriptionInfo.decode(subscription.userData()); assertEquals("localhost:8080", subscriptionInfo.userEndPoint()); } @@ -1300,12 +1292,12 @@ public void shouldMapUserEndPointToTopicPartitions() { partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer1", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, emptyTasks, emptyTasks).encode()) ); - final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); - final ConsumerPartitionAssignor.Assignment consumerAssignment = assignments.get("consumer1"); + final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Assignment consumerAssignment = assignments.get("consumer1"); final AssignmentInfo assignmentInfo = AssignmentInfo.decode(consumerAssignment.userData()); final Set topicPartitions = assignmentInfo.partitionsByHost().get(new HostInfo("localhost", 8080)); assertEquals( @@ -1398,11 +1390,11 @@ public void shouldNotLoopInfinitelyOnMissingMetadataAndShouldNotCreateRelatedTas partitionAssignor.setInternalTopicManager(mockInternalTopicManager); subscriptions.put(client, - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("unknownTopic"), getInfo(uuid, emptyTasks, emptyTasks).encode()) ); - final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); assertThat(mockInternalTopicManager.readyTopics.isEmpty(), equalTo(true)); @@ -1449,18 +1441,18 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() { mockClientSupplier.restoreConsumer)); subscriptions.put("consumer1", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), getInfo(uuid, emptyTasks, emptyTasks).encode()) ); subscriptions.put("consumer2", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), emptyTasks, emptyTasks, "other:9090").encode()) + getInfo(UUID.randomUUID(), getTaskLags(emptyTasks, emptyTasks), "other:9090").encode()) ); final Set allPartitions = mkSet(t1p0, t1p1, t1p2); - final Map assign = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); - final ConsumerPartitionAssignor.Assignment consumer1Assignment = assign.get("consumer1"); + final Map assign = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Assignment consumer1Assignment = assign.get("consumer1"); final AssignmentInfo assignmentInfo = AssignmentInfo.decode(consumer1Assignment.userData()); final Set consumer1ActivePartitions = assignmentInfo.partitionsByHost().get(new HostInfo("localhost", 8080)); @@ -1552,12 +1544,12 @@ public void shouldReturnLowestAssignmentVersionForDifferentSubscriptionVersionsV private void shouldReturnLowestAssignmentVersionForDifferentSubscriptionVersions(final int smallestVersion, final int otherVersion) { subscriptions.put("consumer1", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), getInfo(smallestVersion, UUID.randomUUID(), emptyTasks, emptyTasks, null).encode()) ); subscriptions.put("consumer2", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), getInfo(otherVersion, UUID.randomUUID(), emptyTasks, emptyTasks, null).encode() ) @@ -1567,7 +1559,7 @@ private void shouldReturnLowestAssignmentVersionForDifferentSubscriptionVersions EasyMock.replay(taskManager); streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class); partitionAssignor.configure(configProps()); - final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); assertThat(assignment.size(), equalTo(2)); assertThat(AssignmentInfo.decode(assignment.get("consumer1").userData()).version(), equalTo(smallestVersion)); @@ -1582,7 +1574,7 @@ public void shouldDownGradeSubscriptionToVersion1() { configurePartitionAssignor(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, StreamsConfig.UPGRADE_FROM_0100)); final Set topics = mkSet("topic1"); - final ConsumerPartitionAssignor.Subscription subscription = new ConsumerPartitionAssignor.Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); + final Subscription subscription = new Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); assertThat(SubscriptionInfo.decode(subscription.userData()).version(), equalTo(1)); } @@ -1619,7 +1611,7 @@ private void shouldDownGradeSubscriptionToVersion2(final Object upgradeFromValue configurePartitionAssignor(Collections.singletonMap(StreamsConfig.UPGRADE_FROM_CONFIG, upgradeFromValue)); final Set topics = mkSet("topic1"); - final ConsumerPartitionAssignor.Subscription subscription = new ConsumerPartitionAssignor.Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); + final Subscription subscription = new Subscription(new ArrayList<>(topics), partitionAssignor.subscriptionUserData(topics)); assertThat(SubscriptionInfo.decode(subscription.userData()).version(), equalTo(2)); } @@ -1629,22 +1621,17 @@ public void shouldReturnInterleavedAssignmentWithUnrevokedPartitionsRemovedWhenN builder.addSource(null, "source1", null, null, null, "topic1"); final Set allTasks = mkSet(task0_0, task0_1, task0_2); - final Map allTaskLags = mkMap( - mkEntry(task0_0, 0), - mkEntry(task0_1, 0), - mkEntry(task0_2, 0) - ); subscriptions.put(CONSUMER_1, - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), allTasks, Collections.emptySet(), null).encode(), + getInfo(UUID.randomUUID(), getTaskLags(allTasks, Collections.emptySet()), null).encode(), asList(t1p0, t1p1, t1p2)) ); subscriptions.put(CONSUMER_2, - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), Collections.emptySet(), Collections.emptySet(), null).encode(), + getInfo(UUID.randomUUID(), getTaskLags(Collections.emptySet(), Collections.emptySet()), null).encode(), emptyList()) ); @@ -1653,7 +1640,7 @@ public void shouldReturnInterleavedAssignmentWithUnrevokedPartitionsRemovedWhenN streamsMetadataState = EasyMock.createNiceMock(StreamsMetadataState.class); partitionAssignor.configure(configProps()); - final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); assertThat(assignment.size(), equalTo(2)); @@ -1670,7 +1657,6 @@ public void shouldReturnInterleavedAssignmentWithUnrevokedPartitionsRemovedWhenN 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( @@ -1716,19 +1702,19 @@ public void shouldReturnNormalAssignmentForOldAndFutureInstancesDuringVersionPro partitionAssignor.setInternalTopicManager(new MockInternalTopicManager(streamsConfig, mockClientSupplier.restoreConsumer)); subscriptions.put("consumer1", - new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), activeTasks, standbyTasks, null).encode(), - asList(t1p0, t1p1)) + new Subscription( + Collections.singletonList("topic1"), + getInfo(UUID.randomUUID(), getTaskLags(activeTasks, standbyTasks), null).encode(), + asList(t1p0, t1p1)) ); subscriptions.put("future-consumer", - new ConsumerPartitionAssignor.Subscription( - Collections.singletonList("topic1"), - encodeFutureSubscription(), - Collections.singletonList(t1p2)) + new Subscription( + Collections.singletonList("topic1"), + encodeFutureSubscription(), + Collections.singletonList(t1p2)) ); - final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); assertThat(assignment.size(), equalTo(2)); @@ -1769,13 +1755,13 @@ public void shouldReturnInterleavedAssignmentForOnlyFutureInstancesDuringVersion final Set allTasks = mkSet(task0_0, task0_1, task0_2); subscriptions.put(CONSUMER_1, - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), encodeFutureSubscription(), emptyList()) ); subscriptions.put(CONSUMER_2, - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), encodeFutureSubscription(), emptyList()) @@ -1787,7 +1773,7 @@ public void shouldReturnInterleavedAssignmentForOnlyFutureInstancesDuringVersion final Map props = configProps(); props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 1); partitionAssignor.configure(props); - final Map assignment = + final Map assignment = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); assertThat(assignment.size(), equalTo(2)); @@ -1884,7 +1870,7 @@ public void shouldNotFailOnBranchedMultiLevelRepartitionConnectedTopology() { partitionAssignor.setInternalTopicManager(internalTopicManager); subscriptions.put("consumer10", - new ConsumerPartitionAssignor.Subscription( + new Subscription( topics, getInfo(uuid1, emptyTasks, emptyTasks).encode()) ); @@ -1909,12 +1895,12 @@ private static ByteBuffer encodeFutureSubscription() { private void shouldThrowIfPreVersionProbingSubscriptionAndFutureSubscriptionIsMixed(final int oldVersion) { subscriptions.put("consumer1", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), getInfo(oldVersion, UUID.randomUUID(), emptyTasks, emptyTasks, null).encode()) ); subscriptions.put("future-consumer", - new ConsumerPartitionAssignor.Subscription( + new Subscription( Collections.singletonList("topic1"), encodeFutureSubscription()) ); @@ -1931,13 +1917,13 @@ private void shouldThrowIfPreVersionProbingSubscriptionAndFutureSubscriptionIsMi } } - private static ConsumerPartitionAssignor.Assignment createAssignment(final Map> firstHostState) { + private static Assignment createAssignment(final Map> firstHostState) { final AssignmentInfo info = new AssignmentInfo(LATEST_SUPPORTED_VERSION, emptyList(), emptyMap(), firstHostState, emptyMap(), 0); - return new ConsumerPartitionAssignor.Assignment(emptyList(), info.encode()); + return new Assignment(emptyList(), info.encode()); } private static AssignmentInfo checkAssignment(final Set expectedTopics, - final ConsumerPartitionAssignor.Assignment assignment) { + final Assignment assignment) { // This assumed 1) DefaultPartitionGrouper is used, and 2) there is an only one topic group. @@ -1996,8 +1982,10 @@ private static void assertEquivalentAssignment(final Map> t } } - static Map getTaskLags(final Set activeTasks, final Set cachedTasks) { - return cachedTasks.stream().collect(Collectors.toMap(t -> t, l -> activeTasks.contains(l) ? -1 : 0)); + static Map getTaskLags(final Set activeTasks, final Set standbyTasks) { + final Map taskLags = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> -1)); + taskLags.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0))); + return taskLags; } } 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 e91f89221f7b1..4d7e9d0fb385f 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 @@ -16,6 +16,8 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; + import java.util.HashMap; import java.util.Map; import java.util.stream.Collectors; @@ -35,7 +37,6 @@ public class LegacySubscriptionInfoSerde { private static final Logger log = LoggerFactory.getLogger(LegacySubscriptionInfoSerde.class); - public static final int LATEST_SUPPORTED_VERSION = 6; static final int UNKNOWN = -1; private final int usedVersion; @@ -66,8 +67,8 @@ public LegacySubscriptionInfoSerde(final int version, usedVersion = version; this.latestSupportedVersion = latestSupportedVersion; this.processId = processId; - this.prevTasks = prevTasks == null ? taskLagMapToPrevTaskSet(taskLags) : prevTasks(); - this.standbyTasks = standbyTasks == null ? taskLagMapToStandbyTaskSet(taskLags) : standbyTasks(); + this.prevTasks = prevTasks == null ? taskLagMapToPrevTaskSet(taskLags) : prevTasks; + this.standbyTasks = standbyTasks == null ? taskLagMapToStandbyTaskSet(taskLags) : standbyTasks; // Coerce empty string to null. This was the effect of the serialization logic, anyway. this.userEndPoint = userEndPoint == null || userEndPoint.isEmpty() ? null : userEndPoint; this.taskLags = taskLags == null ? prevAndStandbyTasksToTaskLagMap(prevTasks, standbyTasks) : taskLags; From cac1f396386ecdba75930bff372ce18c8c05abda Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 19 Feb 2020 13:02:52 -0800 Subject: [PATCH 08/24] null check task --- .../kafka/streams/processor/internals/TaskManager.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) 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 4c785f331b177..70b8392b5dd14 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 @@ -491,8 +491,9 @@ private Stream standbyTaskStream() { return tasks.values().stream().filter(t -> !t.isActive()); } - private boolean isActive(final TaskId task) { - return tasks.get(task).isActive(); + private boolean isActive(final TaskId id) { + final Task task = tasks.get(id); + return task != null && task.isActive(); } /** From 72ecf580c005deb312828bde7567f96b170663f2 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 26 Feb 2020 15:31:59 -0800 Subject: [PATCH 09/24] github reivew: main code --- .../processor/internals/StreamsPartitionAssignor.java | 5 ++--- .../kafka/streams/processor/internals/TaskManager.java | 3 +-- .../processor/internals/assignment/SubscriptionInfo.java | 1 + .../org/apache/kafka/streams/tests/StreamsUpgradeTest.java | 2 -- 4 files changed, 4 insertions(+), 7 deletions(-) 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 590429350df59..cad77287a38ed 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 @@ -324,7 +324,6 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final boolean versionProbing; if (futureMetadataVersion == UNKNOWN) { versionProbing = false; - clientMetadataMap.remove(futureId); } else if (minReceivedMetadataVersion >= EARLIEST_PROBEABLE_VERSION) { versionProbing = true; log.info("Received a future (version probing) subscription (version: {})." @@ -563,8 +562,8 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final ClientState state = entry.getValue().state; states.put(uuid, state); - // this is an optimization: we can't decode the future subscription's prev tasks, but we can figure them - // out from the encoded ownedPartitions + // this is an optimization: we can't decode the future subscription info's prev tasks, but we can figure + // them out from the encoded ownedPartitions if (uuid == futureId && !state.ownedPartitions().isEmpty()) { final Set previousActiveTasks = new HashSet<>(); for (final Map.Entry partitionEntry : state.ownedPartitions().entrySet()) { 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 70b8392b5dd14..45dafd2bc474b 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 @@ -51,10 +51,9 @@ import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_LAG; public class TaskManager { - static final int ACTIVE_TASK_SENTINEL_LAG = -1; - // initialize the task list // activeTasks needs to be concurrent as it can be accessed // by QueryableState diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 02fc64f6fbf82..372b6ba9cefbb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -38,6 +38,7 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); + public static final int ACTIVE_TASK_SENTINEL_LAG = -1; static final int UNKNOWN = -1; private final SubscriptionInfoData data; 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 c345211ab628e..57334c2ad4db4 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 @@ -63,8 +63,6 @@ public class StreamsUpgradeTest { - private static final RebalanceProtocol REBALANCE_PROTOCOL = RebalanceProtocol.COOPERATIVE; - @SuppressWarnings("unchecked") public static void main(final String[] args) throws Exception { if (args.length < 1) { From c2da9f7cd5088a3c7f78f6ab77c522a0fc4163d1 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 26 Feb 2020 15:40:56 -0800 Subject: [PATCH 10/24] remove 'paior' from name --- .../internals/assignment/SubscriptionInfo.java | 10 +++++----- .../resources/common/message/SubscriptionInfo.json | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 372b6ba9cefbb..e9f6adc44f56a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -99,11 +99,11 @@ public SubscriptionInfo(final int version, private static void setTaskLagDataFromTaskLagMap(final SubscriptionInfoData data, final Map taskLags) { data.setTaskLags(taskLags.entrySet().stream().map(t -> { - final SubscriptionInfoData.TaskLagPair taskLagPair = new SubscriptionInfoData.TaskLagPair(); - taskLagPair.setTopicGroupId(t.getKey().topicGroupId); - taskLagPair.setPartition(t.getKey().partition); - taskLagPair.setLag(t.getValue()); - return taskLagPair; + final SubscriptionInfoData.TaskLag taskLag = new SubscriptionInfoData.TaskLag(); + taskLag.setTopicGroupId(t.getKey().topicGroupId); + taskLag.setPartition(t.getKey().partition); + taskLag.setLag(t.getValue()); + return taskLag; }).collect(Collectors.toList())); } diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index 3da81f6530be7..f6cc8bfaf6d7c 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -51,7 +51,7 @@ { "name": "taskLags", "versions": "7+", - "type": "[]TaskLagPair" + "type": "[]TaskLag" } ], "commonStructs": [ @@ -72,7 +72,7 @@ ] }, { - "name": "TaskLagPair", + "name": "TaskLag", "versions": "7+", "fields": [ { From a37a8eb48cf12223d052c23319965ca96d6dba9a Mon Sep 17 00:00:00 2001 From: "A. Sophie Blee-Goldman" Date: Wed, 26 Feb 2020 15:54:09 -0800 Subject: [PATCH 11/24] Github suggestions Co-Authored-By: Bruno Cadonna --- .../internals/assignment/SubscriptionInfoTest.java | 14 ++++++++++---- 1 file changed, 10 insertions(+), 4 deletions(-) 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 b9012e0be17ee..4a14a63f4a899 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 @@ -40,7 +40,13 @@ public class SubscriptionInfoTest { private static final Set STANDBY_TASKS = new HashSet<>(Arrays.asList( new TaskId(1, 1), new TaskId(2, 0))); - private static final Map TASK_LAGS = new HashMap<>(); + private static final Map TASK_LAGS = mkMap( + mkEntry(new TaskId(0, 0), -1), + mkEntry(new TaskId(0, 1), -1), + mkEntry(new TaskId(1, 0), -1), + mkEntry(new TaskId(1, 1), 0), + mkEntry(new TaskId(2, 0), 10) + ); static { TASK_LAGS.put(new TaskId(0, 0), -1); TASK_LAGS.put(new TaskId(0, 1), -1); @@ -290,14 +296,14 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { public void shouldEncodeAndDecodeVersion7() { final SubscriptionInfo info = new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); - assertEquals(info, SubscriptionInfo.decode(info.encode())); + assertThat(SubscriptionInfo.decode(info.encode()), is(info)); } @Test public void shouldConvertTaskLagMapToTaskSetsForOlderVersion() { final SubscriptionInfo info = new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); - assertEquals(info.prevTasks(), ACTIVE_TASKS); - assertEquals(info.standbyTasks(), STANDBY_TASKS); + assertThat(info.prevTasks(), is(ACTIVE_TASKS)); + assertThat(info.standbyTasks(), is(STANDBY_TASKS)); } private static ByteBuffer encodeFutureVersion() { From 40e20e352f46b7be84058c9859004a0106592eef Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 28 Feb 2020 18:32:37 -0500 Subject: [PATCH 12/24] use offset sum instead of lag --- .../processor/internals/TaskManager.java | 4 +- .../assignment/SubscriptionInfo.java | 55 ++++++++++--------- .../common/message/SubscriptionInfo.json | 8 +-- .../assignment/SubscriptionInfoTest.java | 50 +++++++++-------- 4 files changed, 61 insertions(+), 56 deletions(-) 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 45dafd2bc474b..dfb3273538d21 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 @@ -51,7 +51,7 @@ import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; -import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_LAG; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; public class TaskManager { // initialize the task list @@ -364,7 +364,7 @@ public Map getTaskLags() { for (final TaskId id : tasksOnLocalStorage()) { if (isActive(id)) { - taskLags.put(id, ACTIVE_TASK_SENTINEL_LAG); + taskLags.put(id, ACTIVE_TASK_SENTINEL_OFFSET); } else { taskLags.put(id, 0); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index e9f6adc44f56a..9496beddd96cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData; +import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.TaskOffsetSum; import org.apache.kafka.streams.processor.TaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,13 +39,13 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); - public static final int ACTIVE_TASK_SENTINEL_LAG = -1; + public static final int ACTIVE_TASK_SENTINEL_OFFSET = -1; static final int UNKNOWN = -1; private final SubscriptionInfoData data; private Set prevTasksCache = null; private Set standbyTasksCache = null; - private Map taskLagsCache = null; + private Map taskOffsetSumsCache = null; static { // Just statically check to make sure that the generated code always stays in sync with the overall protocol @@ -74,7 +75,7 @@ public SubscriptionInfo(final int version, final int latestSupportedVersion, final UUID processId, final String userEndPoint, - final Map taskLags) { + final Map taskOffsetSums) { validateVersions(version, latestSupportedVersion); final SubscriptionInfoData data = new SubscriptionInfoData(); data.setVersion(version); @@ -89,34 +90,34 @@ public SubscriptionInfo(final int version, data.setLatestSupportedVersion(latestSupportedVersion); } if (version >= 7) { - setTaskLagDataFromTaskLagMap(data, taskLags); + setTaskOffsetSumDataFromTaskOffsetSumMap(data, taskOffsetSums); } else { - setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags); + setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums); } this.data = data; } - private static void setTaskLagDataFromTaskLagMap(final SubscriptionInfoData data, - final Map taskLags) { - data.setTaskLags(taskLags.entrySet().stream().map(t -> { - final SubscriptionInfoData.TaskLag taskLag = new SubscriptionInfoData.TaskLag(); - taskLag.setTopicGroupId(t.getKey().topicGroupId); - taskLag.setPartition(t.getKey().partition); - taskLag.setLag(t.getValue()); - return taskLag; + private static void setTaskOffsetSumDataFromTaskOffsetSumMap(final SubscriptionInfoData data, + final Map taskOffsetSums) { + data.setTaskOffsetSums(taskOffsetSums.entrySet().stream().map(t -> { + final SubscriptionInfoData.TaskOffsetSum taskOffsetSum = new SubscriptionInfoData.TaskOffsetSum(); + taskOffsetSum.setTopicGroupId(t.getKey().topicGroupId); + taskOffsetSum.setPartition(t.getKey().partition); + taskOffsetSum.setOffsetSum(t.getValue()); + return taskOffsetSum; }).collect(Collectors.toList())); } - private static void setPrevAndStandbySetsFromParsedTaskLagMap(final SubscriptionInfoData data, - final Map taskLags) { + private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final SubscriptionInfoData data, + final Map taskOffsetSums) { final Set prevTasks = new HashSet<>(); final Set standbyTasks = new HashSet<>(); - for (final Map.Entry taskLagEntry : taskLags.entrySet()) { - if (taskLagEntry.getValue() == -1) { - prevTasks.add(taskLagEntry.getKey()); + for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { + if (taskOffsetSum.getValue() == -1) { + prevTasks.add(taskOffsetSum.getKey()); } else { - standbyTasks.add(taskLagEntry.getKey()); + standbyTasks.add(taskOffsetSum.getKey()); } } @@ -155,7 +156,7 @@ public Set prevTasks() { if (prevTasksCache == null) { // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { - setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); + setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums()); } prevTasksCache = Collections.unmodifiableSet( data.prevTasks() @@ -171,7 +172,7 @@ public Set standbyTasks() { if (standbyTasksCache == null) { // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { - setPrevAndStandbySetsFromParsedTaskLagMap(data, taskLags()); + setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums()); } standbyTasksCache = Collections.unmodifiableSet( data.standbyTasks() @@ -183,15 +184,15 @@ public Set standbyTasks() { return standbyTasksCache; } - public Map taskLags() { - if (taskLagsCache == null) { - taskLagsCache = Collections.unmodifiableMap( - data.taskLags() + public Map taskOffsetSums() { + if (taskOffsetSumsCache == null) { + taskOffsetSumsCache = Collections.unmodifiableMap( + data.taskOffsetSums() .stream() - .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), l -> l.lag())) + .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), TaskOffsetSum::offsetSum)) ); } - return taskLagsCache; + return taskOffsetSumsCache; } public String userEndPoint() { diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index f6cc8bfaf6d7c..058d157fdd51f 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -49,9 +49,9 @@ "type": "bytes" }, { - "name": "taskLags", + "name": "taskOffsetSums", "versions": "7+", - "type": "[]TaskLag" + "type": "[]TaskOffsetSum" } ], "commonStructs": [ @@ -72,7 +72,7 @@ ] }, { - "name": "TaskLag", + "name": "TaskOffsetSum", "versions": "7+", "fields": [ { @@ -86,7 +86,7 @@ "type": "int32" }, { - "name": "lag", + "name": "offsetSum", "versions": "1+", "type": "int32" } 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 4a14a63f4a899..c53b304b88cb9 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 @@ -16,7 +16,6 @@ */ package org.apache.kafka.streams.processor.internals.assignment; -import java.util.HashMap; import java.util.Map; import org.apache.kafka.streams.processor.TaskId; import org.junit.Test; @@ -27,7 +26,11 @@ import java.util.Set; import java.util.UUID; +import static org.apache.kafka.common.utils.Utils.mkEntry; +import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -40,7 +43,7 @@ public class SubscriptionInfoTest { private static final Set STANDBY_TASKS = new HashSet<>(Arrays.asList( new TaskId(1, 1), new TaskId(2, 0))); - private static final Map TASK_LAGS = mkMap( + private static final Map TASK_OFFSET_SUMS = mkMap( mkEntry(new TaskId(0, 0), -1), mkEntry(new TaskId(0, 1), -1), mkEntry(new TaskId(1, 0), -1), @@ -48,11 +51,11 @@ public class SubscriptionInfoTest { mkEntry(new TaskId(2, 0), 10) ); static { - TASK_LAGS.put(new TaskId(0, 0), -1); - TASK_LAGS.put(new TaskId(0, 1), -1); - TASK_LAGS.put(new TaskId(1, 0), -1); - TASK_LAGS.put(new TaskId(1, 1), 0); - TASK_LAGS.put(new TaskId(2, 0), 10); + TASK_OFFSET_SUMS.put(new TaskId(0, 0), -1); + TASK_OFFSET_SUMS.put(new TaskId(0, 1), -1); + TASK_OFFSET_SUMS.put(new TaskId(1, 0), -1); + TASK_OFFSET_SUMS.put(new TaskId(1, 1), 0); + TASK_OFFSET_SUMS.put(new TaskId(2, 0), 10); } private final static String IGNORED_USER_ENDPOINT = "ignoredUserEndpoint:80"; @@ -64,7 +67,7 @@ public void shouldThrowForUnknownVersion1() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); } @@ -75,7 +78,7 @@ public void shouldThrowForUnknownVersion2() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); } @@ -86,7 +89,7 @@ public void shouldEncodeAndDecodeVersion1() { LATEST_SUPPORTED_VERSION, processId, IGNORED_USER_ENDPOINT, - TASK_LAGS + TASK_OFFSET_SUMS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(1, decoded.version()); @@ -104,7 +107,7 @@ public void generatedVersion1ShouldBeDecodableByLegacyLogic() { 1234, processId, "ignoreme", - TASK_LAGS + TASK_OFFSET_SUMS ); final ByteBuffer buffer = info.encode(); @@ -146,7 +149,7 @@ public void shouldEncodeAndDecodeVersion2() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(2, decoded.version()); @@ -164,7 +167,7 @@ public void generatedVersion2ShouldBeDecodableByLegacyLogic() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); final ByteBuffer buffer = info.encode(); @@ -207,7 +210,7 @@ public void shouldEncodeAndDecodeVersion3And4() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); final SubscriptionInfo decoded = SubscriptionInfo.decode(info.encode()); assertEquals(version, decoded.version()); @@ -227,7 +230,7 @@ public void generatedVersion3And4ShouldBeDecodableByLegacyLogic() { LATEST_SUPPORTED_VERSION, processId, "localhost:80", - TASK_LAGS + TASK_OFFSET_SUMS ); final ByteBuffer buffer = info.encode(); @@ -269,7 +272,7 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { @Test public void shouldEncodeAndDecodeVersion5() { final SubscriptionInfo info = - new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); + new SubscriptionInfo(5, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_OFFSET_SUMS); assertEquals(info, SubscriptionInfo.decode(info.encode())); } @@ -286,29 +289,30 @@ public void shouldEncodeAndDecodeSmallerLatestSupportedVersion() { final int latestSupportedVersion = LATEST_SUPPORTED_VERSION - 1; final SubscriptionInfo info = - new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_LAGS); + new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_OFFSET_SUMS); final SubscriptionInfo expectedInfo = - new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_LAGS); + new SubscriptionInfo(usedVersion, latestSupportedVersion, processId, "localhost:80", TASK_OFFSET_SUMS); assertEquals(expectedInfo, SubscriptionInfo.decode(info.encode())); } @Test public void shouldEncodeAndDecodeVersion7() { final SubscriptionInfo info = - new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); - assertThat(SubscriptionInfo.decode(info.encode()), is(info)); + new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_OFFSET_SUMS); + assertThat(info, is(SubscriptionInfo.decode(info.encode()))); } @Test - public void shouldConvertTaskLagMapToTaskSetsForOlderVersion() { - final SubscriptionInfo info = new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_LAGS); + public void shouldConvertTaskOffsetSumMapToTaskSetsForOlderVersion() { + final SubscriptionInfo info = + new SubscriptionInfo(7, LATEST_SUPPORTED_VERSION, processId, "localhost:80", TASK_OFFSET_SUMS); assertThat(info.prevTasks(), is(ACTIVE_TASKS)); assertThat(info.standbyTasks(), is(STANDBY_TASKS)); } private static ByteBuffer encodeFutureVersion() { final ByteBuffer buf = ByteBuffer.allocate(4 /* used version */ - + 4 /* supported version */); + + 4 /* supported version */); buf.putInt(LATEST_SUPPORTED_VERSION + 1); buf.putInt(LATEST_SUPPORTED_VERSION + 1); buf.rewind(); From 7d8bdc21fa4f36c01f54884f8bd2fada0b135da2 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Fri, 28 Feb 2020 18:36:50 -0500 Subject: [PATCH 13/24] fix names --- .../internals/StreamsPartitionAssignor.java | 2 +- .../processor/internals/TaskManager.java | 10 ++--- .../StreamsPartitionAssignorTest.java | 40 +++++++++---------- .../streams/tests/StreamsUpgradeTest.java | 6 +-- 4 files changed, 29 insertions(+), 29 deletions(-) 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 cad77287a38ed..14b3fd2c6c20e 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 @@ -226,7 +226,7 @@ public ByteBuffer subscriptionUserData(final Set topics) { LATEST_SUPPORTED_VERSION, taskManager.processId(), userEndPoint, - taskManager.getTaskLags()) + taskManager.getTaskOffsetSums()) .encode(); } 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 dfb3273538d21..24ab9e2a2277c 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 @@ -359,17 +359,17 @@ void handleLostAll() { /** * @return Map from task id to that task's overall lag across all state stores */ - public Map getTaskLags() { - final Map taskLags = new HashMap<>(); + public Map getTaskOffsetSums() { + final Map taskOffsetSums = new HashMap<>(); for (final TaskId id : tasksOnLocalStorage()) { if (isActive(id)) { - taskLags.put(id, ACTIVE_TASK_SENTINEL_OFFSET); + taskOffsetSums.put(id, ACTIVE_TASK_SENTINEL_OFFSET); } else { - taskLags.put(id, 0); + taskOffsetSums.put(id, 0); } } - return taskLags; + return taskOffsetSums; } /** 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 1a76662780200..96059031b7709 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 @@ -205,7 +205,7 @@ private void createMockTaskManager(final Set prevTasks, final InternalTopologyBuilder builder) { taskManager = EasyMock.createNiceMock(TaskManager.class); EasyMock.expect(taskManager.builder()).andReturn(builder).anyTimes(); - EasyMock.expect(taskManager.getTaskLags()).andReturn(getTaskLags(prevTasks, standbyTasks)).anyTimes(); + EasyMock.expect(taskManager.getTaskOffsetSums()).andReturn(getTaskOffsetSums(prevTasks, standbyTasks)).anyTimes(); EasyMock.expect(taskManager.processId()).andReturn(processId).anyTimes(); builder.setApplicationId(APPLICATION_ID); builder.buildTopology(); @@ -228,7 +228,7 @@ private static SubscriptionInfo getInfo(final int version, final Set standbyTasks, final String userEndPoint) { return new SubscriptionInfo( - version, LATEST_SUPPORTED_VERSION, processId, userEndPoint, getTaskLags(prevTasks, standbyTasks)); + version, LATEST_SUPPORTED_VERSION, processId, userEndPoint, getTaskOffsetSums(prevTasks, standbyTasks)); } private static SubscriptionInfo getInfo(final UUID processId, @@ -240,7 +240,7 @@ private static SubscriptionInfo getInfo(final UUID processId, private static SubscriptionInfo getInfo(final UUID processId, final Set prevTasks, final Set standbyTasks) { return new SubscriptionInfo( - LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, USER_END_POINT, getTaskLags(prevTasks, standbyTasks)); + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, USER_END_POINT, getTaskOffsetSums(prevTasks, standbyTasks)); } @Test @@ -452,7 +452,7 @@ public void testEagerSubscription() { assertEquals(asList("topic1", "topic2"), subscription.topics()); - final SubscriptionInfo info = getInfo(processId, getTaskLags(prevTasks, standbyTasks), null); + final SubscriptionInfo info = getInfo(processId, getTaskOffsetSums(prevTasks, standbyTasks), null); assertEquals(info, SubscriptionInfo.decode(subscription.userData())); } @@ -485,7 +485,7 @@ public void testCooperativeSubscription() { final Set standbyTasks = new HashSet<>(cachedTasks); standbyTasks.removeAll(prevTasks); - final SubscriptionInfo info = getInfo(processId, getTaskLags(prevTasks, cachedTasks), null); + final SubscriptionInfo info = getInfo(processId, getTaskOffsetSums(prevTasks, cachedTasks), null); assertEquals(info, SubscriptionInfo.decode(subscription.userData())); } @@ -517,17 +517,17 @@ public void testAssignBasic() { subscriptions.put("consumer10", new Subscription( topics, - getInfo(uuid1, getTaskLags(prevTasks10, standbyTasks10), USER_END_POINT).encode() + getInfo(uuid1, getTaskOffsetSums(prevTasks10, standbyTasks10), USER_END_POINT).encode() )); subscriptions.put("consumer11", new Subscription( topics, - getInfo(uuid1, getTaskLags(prevTasks11, standbyTasks11), USER_END_POINT).encode() + getInfo(uuid1, getTaskOffsetSums(prevTasks11, standbyTasks11), USER_END_POINT).encode() )); subscriptions.put("consumer20", new Subscription( topics, - getInfo(uuid2, getTaskLags(prevTasks20, standbyTasks20), USER_END_POINT).encode() + getInfo(uuid2, getTaskOffsetSums(prevTasks20, standbyTasks20), USER_END_POINT).encode() )); final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); @@ -904,11 +904,11 @@ public void testAssignWithStandbyReplicasAndStatelessTasks() { subscriptions.put("consumer10", new Subscription( topics, - getInfo(uuid1, getTaskLags(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); + getInfo(uuid1, getTaskOffsetSums(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); subscriptions.put("consumer20", new Subscription( topics, - getInfo(uuid2, getTaskLags(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); + getInfo(uuid2, getTaskOffsetSums(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); @@ -943,11 +943,11 @@ public void testAssignWithStandbyReplicasAndLoggingDisabled() { subscriptions.put("consumer10", new Subscription( topics, - getInfo(uuid1, getTaskLags(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); + getInfo(uuid1, getTaskOffsetSums(mkSet(task0_0), Collections.emptySet()), "any:9096").encode())); subscriptions.put("consumer20", new Subscription( topics, - getInfo(uuid2, getTaskLags(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); + getInfo(uuid2, getTaskOffsetSums(mkSet(task0_2), Collections.emptySet()), "any:9097").encode())); final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); @@ -999,15 +999,15 @@ public void testAssignWithStandbyReplicas() { subscriptions.put("consumer10", new Subscription( topics, - getInfo(uuid1, getTaskLags(prevTasks00, standbyTasks01), "any:9096").encode())); + getInfo(uuid1, getTaskOffsetSums(prevTasks00, standbyTasks01), "any:9096").encode())); subscriptions.put("consumer11", new Subscription( topics, - getInfo(uuid1, getTaskLags(prevTasks01, standbyTasks02), "any:9096").encode())); + getInfo(uuid1, getTaskOffsetSums(prevTasks01, standbyTasks02), "any:9096").encode())); subscriptions.put("consumer20", new Subscription( topics, - getInfo(uuid2, getTaskLags(prevTasks02, standbyTasks00), "any:9097").encode())); + getInfo(uuid2, getTaskOffsetSums(prevTasks02, standbyTasks00), "any:9097").encode())); final Map assignments = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); @@ -1448,7 +1448,7 @@ public void shouldNotAddStandbyTaskPartitionsToPartitionsForHost() { subscriptions.put("consumer2", new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), getTaskLags(emptyTasks, emptyTasks), "other:9090").encode()) + getInfo(UUID.randomUUID(), getTaskOffsetSums(emptyTasks, emptyTasks), "other:9090").encode()) ); final Set allPartitions = mkSet(t1p0, t1p1, t1p2); final Map assign = partitionAssignor.assign(metadata, new GroupSubscription(subscriptions)).groupAssignment(); @@ -1625,13 +1625,13 @@ public void shouldReturnInterleavedAssignmentWithUnrevokedPartitionsRemovedWhenN subscriptions.put(CONSUMER_1, new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), getTaskLags(allTasks, Collections.emptySet()), null).encode(), + getInfo(UUID.randomUUID(), getTaskOffsetSums(allTasks, Collections.emptySet()), null).encode(), asList(t1p0, t1p1, t1p2)) ); subscriptions.put(CONSUMER_2, new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), getTaskLags(Collections.emptySet(), Collections.emptySet()), null).encode(), + getInfo(UUID.randomUUID(), getTaskOffsetSums(Collections.emptySet(), Collections.emptySet()), null).encode(), emptyList()) ); @@ -1704,7 +1704,7 @@ public void shouldReturnNormalAssignmentForOldAndFutureInstancesDuringVersionPro subscriptions.put("consumer1", new Subscription( Collections.singletonList("topic1"), - getInfo(UUID.randomUUID(), getTaskLags(activeTasks, standbyTasks), null).encode(), + getInfo(UUID.randomUUID(), getTaskOffsetSums(activeTasks, standbyTasks), null).encode(), asList(t1p0, t1p1)) ); subscriptions.put("future-consumer", @@ -1982,7 +1982,7 @@ private static void assertEquivalentAssignment(final Map> t } } - static Map getTaskLags(final Set activeTasks, final Set standbyTasks) { + static Map getTaskOffsetSums(final Set activeTasks, final Set standbyTasks) { final Map taskLags = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> -1)); taskLags.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0))); return taskLags; 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 57334c2ad4db4..0ae746648ed50 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 @@ -152,14 +152,14 @@ public ByteBuffer subscriptionUserData(final Set topics) { LATEST_SUPPORTED_VERSION + 1, taskManager.processId(), userEndPoint(), - taskManager.getTaskLags() + taskManager.getTaskOffsetSums() ).encode(); } else { return new FutureSubscriptionInfo( usedSubscriptionMetadataVersion, taskManager.processId(), userEndPoint(), - taskManager.getTaskLags()) + taskManager.getTaskOffsetSums()) .encode(); } } @@ -248,7 +248,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, info.processId(), - info.userEndPoint(), taskManger().getTaskLags()) + info.userEndPoint(), taskManger().getTaskOffsetSums()) .encode(), subscription.ownedPartitions() )); From e4e941f8dacb35590b9ff247a0eed9bf8a54a28d Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Sat, 29 Feb 2020 01:18:21 -0500 Subject: [PATCH 14/24] github review --- .../processor/internals/StreamsPartitionAssignor.java | 9 +++++---- .../kafka/streams/processor/internals/TaskManager.java | 5 ++--- .../internals/assignment/SubscriptionInfo.java | 3 ++- .../resources/common/message/SubscriptionInfo.json | 6 +++--- .../streams/processor/internals/TaskManagerTest.java | 6 +++--- .../apache/kafka/streams/tests/StreamsUpgradeTest.java | 10 +++++----- 6 files changed, 20 insertions(+), 19 deletions(-) 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 14b3fd2c6c20e..60739891a9a43 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 @@ -562,9 +562,10 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr final ClientState state = entry.getValue().state; states.put(uuid, state); - // this is an optimization: we can't decode the future subscription info's prev tasks, but we can figure - // them out from the encoded ownedPartitions - if (uuid == futureId && !state.ownedPartitions().isEmpty()) { + // there are two cases where we need to construct the prevTasks from the ownedPartitions: + // 1) COOPERATIVE clients on version 2.4-2.5 do not encode active tasks and rely on ownedPartitions instead + // 2) future clientduring version probing: we can't decode the future subscription info's prev tasks + if (!state.ownedPartitions().isEmpty() && (uuid == futureId ||state.prevActiveTasks().isEmpty())) { final Set previousActiveTasks = new HashSet<>(); for (final Map.Entry partitionEntry : state.ownedPartitions().entrySet()) { final TopicPartition tp = partitionEntry.getKey(); @@ -1264,7 +1265,7 @@ protected String userEndPoint() { return userEndPoint; } - protected TaskManager taskManger() { + protected TaskManager taskManager() { return taskManager; } 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 24ab9e2a2277c..ac80e943bb84b 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 @@ -117,8 +117,7 @@ boolean isRebalanceInProgress() { return rebalanceInProgress; } - // visible for testing - public void handleRebalanceStart(final Set subscribedTopics) { + void handleRebalanceStart(final Set subscribedTopics) { builder.addSubscribedTopicsFromMetadata(subscribedTopics, logPrefix); rebalanceInProgress = true; @@ -376,7 +375,7 @@ public Map getTaskOffsetSums() { * Returns ids of tasks whose states are kept on the local storage. This includes active, standby, and previously * assigned but not yet cleaned up tasks */ - Set tasksOnLocalStorage() { + private Set tasksOnLocalStorage() { // A client could contain some inactive tasks whose states are still kept on the local storage in the following scenarios: // 1) the client is actively maintaining standby tasks by maintaining their states from the change log. // 2) the client has just got some tasks migrated out of itself to other clients while these task states diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 9496beddd96cd..53e99186ee7d5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -39,6 +39,7 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); + // encode -1 to differentiate between an active task and a standby task that has caught up to the same offsets public static final int ACTIVE_TASK_SENTINEL_OFFSET = -1; static final int UNKNOWN = -1; @@ -114,7 +115,7 @@ private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Subscr final Set standbyTasks = new HashSet<>(); for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { - if (taskOffsetSum.getValue() == -1) { + if (taskOffsetSum.getValue() == ACTIVE_TASK_SENTINEL_OFFSET) { prevTasks.add(taskOffsetSum.getKey()); } else { standbyTasks.add(taskOffsetSum.getKey()); diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index 058d157fdd51f..ad4244f8cebb6 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -77,17 +77,17 @@ "fields": [ { "name": "topicGroupId", - "versions": "1+", + "versions": "7+", "type": "int32" }, { "name": "partition", - "versions": "1+", + "versions": "7+", "type": "int32" }, { "name": "offsetSum", - "versions": "1+", + "versions": "7+", "type": "int32" } ] diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 3f38a11e38004..0dd4d203ec62e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -153,7 +153,7 @@ public void shouldIdempotentlyUpdateSubscriptionFromActiveAssignment() { } @Test - public void shouldReturnCachedTaskIdsFromDirectory() throws IOException { + public void shouldReturnOffsetsForAllCachedTaskIdsFromDirectory() throws IOException { final File[] taskFolders = asList(testFolder.newFolder("0_1"), testFolder.newFolder("0_2"), testFolder.newFolder("0_3"), @@ -169,11 +169,11 @@ public void shouldReturnCachedTaskIdsFromDirectory() throws IOException { replay(activeTaskCreator, stateDirectory); - final Set tasks = taskManager.tasksOnLocalStorage(); + final Map taskOffsetSums = taskManager.getTaskOffsetSums(); verify(activeTaskCreator, stateDirectory); - assertThat(tasks, equalTo(mkSet(taskId01, taskId02, new TaskId(1, 1)))); + assertThat(taskOffsetSums.keySet(), equalTo(mkSet(taskId01, taskId02, new TaskId(1, 1)))); } @Test 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 0ae746648ed50..0aa9832c3dd08 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 @@ -20,7 +20,6 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor; -import org.apache.kafka.clients.consumer.ConsumerPartitionAssignor.RebalanceProtocol; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.TopicPartition; @@ -139,12 +138,13 @@ public void configure(final Map configs) { @Override public ByteBuffer subscriptionUserData(final Set topics) { + super.subscriptionUserData(topics); + // Adds the following information to subscription // 1. Client UUID (a unique id assigned to an instance of KafkaStreams) // 2. Task ids of previously running tasks // 3. Task ids of valid local states on the client's state directory. - final TaskManager taskManager = taskManger(); - taskManager.handleRebalanceStart(topics); + final TaskManager taskManager = taskManager(); if (usedSubscriptionMetadataVersion <= LATEST_SUPPORTED_VERSION) { return new SubscriptionInfo( @@ -203,7 +203,7 @@ public void onAssignment(final ConsumerPartitionAssignor.Assignment assignment, final Map> activeTasks = getActiveTasks(partitions, info); - final TaskManager taskManager = taskManger(); + final TaskManager taskManager = taskManager(); taskManager.handleAssignment(activeTasks, info.standbyTasks()); usedSubscriptionMetadataVersionPeek.set(usedSubscriptionMetadataVersion); } @@ -248,7 +248,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, info.processId(), - info.userEndPoint(), taskManger().getTaskOffsetSums()) + info.userEndPoint(), taskManager().getTaskOffsetSums()) .encode(), subscription.ownedPartitions() )); From 0fe088a5e16694ba008c4f16f4face445bda28b6 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Sat, 29 Feb 2020 18:10:03 -0500 Subject: [PATCH 15/24] bump version in VP system test --- .../kafkatest/tests/streams/streams_upgrade_test.py | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/tests/kafkatest/tests/streams/streams_upgrade_test.py b/tests/kafkatest/tests/streams/streams_upgrade_test.py index 505ab82933a33..ab4f0af415cce 100644 --- a/tests/kafkatest/tests/streams/streams_upgrade_test.py +++ b/tests/kafkatest/tests/streams/streams_upgrade_test.py @@ -525,19 +525,19 @@ def do_rolling_bounce(self, processor, counter, current_generation): monitors[second_other_processor] = second_other_monitor if len(self.old_processors) > 0: - log_monitor.wait_until("Sent a version 7 subscription and got version 6 assignment back (successful version probing). Downgrade subscription metadata to commonly supported version 6 and trigger new rebalance.", + log_monitor.wait_until("Sent a version 8 subscription and got version 7 assignment back (successful version probing). Downgrade subscription metadata to commonly supported version 7 and trigger new rebalance.", timeout_sec=60, err_msg="Could not detect 'successful version probing' at upgrading node " + str(node.account)) else: - log_monitor.wait_until("Sent a version 7 subscription and got version 6 assignment back (successful version probing). Downgrade subscription metadata to commonly supported version 7 and trigger new rebalance.", + log_monitor.wait_until("Sent a version 8 subscription and got version 7 assignment back (successful version probing). Downgrade subscription metadata to commonly supported version 8 and trigger new rebalance.", timeout_sec=60, err_msg="Could not detect 'successful version probing with upgraded leader' at upgrading node " + str(node.account)) - first_other_monitor.wait_until("Sent a version 6 subscription and group.s latest commonly supported version is 7 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 7 for next rebalance.", + first_other_monitor.wait_until("Sent a version 7 subscription and group.s latest commonly supported version is 8 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 8 for next rebalance.", timeout_sec=60, - err_msg="Never saw output 'Upgrade metadata to version 7' on" + str(first_other_node.account)) - second_other_monitor.wait_until("Sent a version 6 subscription and group.s latest commonly supported version is 7 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 7 for next rebalance.", + err_msg="Never saw output 'Upgrade metadata to version 8' on" + str(first_other_node.account)) + second_other_monitor.wait_until("Sent a version 7 subscription and group.s latest commonly supported version is 8 (successful version probing and end of rolling upgrade). Upgrading subscription metadata version to 8 for next rebalance.", timeout_sec=60, - err_msg="Never saw output 'Upgrade metadata to version 7' on" + str(second_other_node.account)) + err_msg="Never saw output 'Upgrade metadata to version 8' on" + str(second_other_node.account)) log_monitor.wait_until("Version probing detected. Rejoining the consumer group to trigger a new rebalance.", timeout_sec=60, From 361e4496245017a7301d481934d6a8aafc8f0d80 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 4 Mar 2020 13:54:26 -0500 Subject: [PATCH 16/24] review --- .../processor/internals/TaskManager.java | 6 +-- .../assignment/SubscriptionInfo.java | 32 ++++++++++---- .../common/message/SubscriptionInfo.json | 2 +- .../processor/internals/TaskManagerTest.java | 4 +- .../LegacySubscriptionInfoSerde.java | 43 +++---------------- .../assignment/SubscriptionInfoTest.java | 29 +++++-------- .../streams/tests/StreamsUpgradeTest.java | 37 ++++++++-------- 7 files changed, 66 insertions(+), 87 deletions(-) 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 ac80e943bb84b..25d32cdb5677a 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 @@ -358,14 +358,14 @@ void handleLostAll() { /** * @return Map from task id to that task's overall lag across all state stores */ - public Map getTaskOffsetSums() { - final Map taskOffsetSums = new HashMap<>(); + public Map getTaskOffsetSums() { + final Map taskOffsetSums = new HashMap<>(); for (final TaskId id : tasksOnLocalStorage()) { if (isActive(id)) { taskOffsetSums.put(id, ACTIVE_TASK_SENTINEL_OFFSET); } else { - taskOffsetSums.put(id, 0); + taskOffsetSums.put(id, 0L); } } return taskOffsetSums; diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 53e99186ee7d5..38b84c772dfa6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -39,14 +39,14 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); - // encode -1 to differentiate between an active task and a standby task that has caught up to the same offsets - public static final int ACTIVE_TASK_SENTINEL_OFFSET = -1; + // encode running active tasks as -1 to skip computing their offset sum since we know they are caught up + public static final long ACTIVE_TASK_SENTINEL_OFFSET = -1; static final int UNKNOWN = -1; private final SubscriptionInfoData data; private Set prevTasksCache = null; private Set standbyTasksCache = null; - private Map taskOffsetSumsCache = null; + private Map taskOffsetSumsCache = null; static { // Just statically check to make sure that the generated code always stays in sync with the overall protocol @@ -76,7 +76,7 @@ public SubscriptionInfo(final int version, final int latestSupportedVersion, final UUID processId, final String userEndPoint, - final Map taskOffsetSums) { + final Map taskOffsetSums) { validateVersions(version, latestSupportedVersion); final SubscriptionInfoData data = new SubscriptionInfoData(); data.setVersion(version); @@ -99,7 +99,7 @@ public SubscriptionInfo(final int version, } private static void setTaskOffsetSumDataFromTaskOffsetSumMap(final SubscriptionInfoData data, - final Map taskOffsetSums) { + final Map taskOffsetSums) { data.setTaskOffsetSums(taskOffsetSums.entrySet().stream().map(t -> { final SubscriptionInfoData.TaskOffsetSum taskOffsetSum = new SubscriptionInfoData.TaskOffsetSum(); taskOffsetSum.setTopicGroupId(t.getKey().topicGroupId); @@ -110,11 +110,11 @@ private static void setTaskOffsetSumDataFromTaskOffsetSumMap(final SubscriptionI } private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final SubscriptionInfoData data, - final Map taskOffsetSums) { + final Map taskOffsetSums) { final Set prevTasks = new HashSet<>(); final Set standbyTasks = new HashSet<>(); - for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { + for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { if (taskOffsetSum.getValue() == ACTIVE_TASK_SENTINEL_OFFSET) { prevTasks.add(taskOffsetSum.getKey()); } else { @@ -185,7 +185,7 @@ public Set standbyTasks() { return standbyTasksCache; } - public Map taskOffsetSums() { + public Map taskOffsetSums() { if (taskOffsetSumsCache == null) { taskOffsetSumsCache = Collections.unmodifiableMap( data.taskOffsetSums() @@ -202,6 +202,22 @@ public String userEndPoint() { : new String(data.userEndPoint(), StandardCharsets.UTF_8); } + public static Set getActiveTasksFromTaskOffsetSumMap(final Map taskOffsetSums) { + return taskOffsetSumMapToTaskSet(taskOffsetSums, true); + } + + public static Set getStandbyTasksFromTaskOffsetSumMap(final Map taskOffsetSums) { + return taskOffsetSumMapToTaskSet(taskOffsetSums, false); + } + + private static Set taskOffsetSumMapToTaskSet(final Map taskOffsetSums, + final boolean getActiveTasks) { + return taskOffsetSums.entrySet().stream() + .filter(t -> getActiveTasks == (t.getValue() == ACTIVE_TASK_SENTINEL_OFFSET)) + .map(Map.Entry::getKey) + .collect(Collectors.toSet()); + } + /** * @throws TaskAssignmentException if method fails to encode the data */ diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index ad4244f8cebb6..21716363bb065 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -88,7 +88,7 @@ { "name": "offsetSum", "versions": "7+", - "type": "int32" + "type": "int64" } ] } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java index 0dd4d203ec62e..7ff7d4aa15a9b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java @@ -169,7 +169,7 @@ public void shouldReturnOffsetsForAllCachedTaskIdsFromDirectory() throws IOExcep replay(activeTaskCreator, stateDirectory); - final Map taskOffsetSums = taskManager.getTaskOffsetSums(); + final Map taskOffsetSums = taskManager.getTaskOffsetSums(); verify(activeTaskCreator, stateDirectory); @@ -177,7 +177,7 @@ public void shouldReturnOffsetsForAllCachedTaskIdsFromDirectory() throws IOExcep } @Test - public void shouldCloseActiveUnAssignedSuspendedTasksWhenClosingRevokedTasks() { + public void shouldCloseActiveUnassignedSuspendedTasksWhenClosingRevokedTasks() { final Task task00 = new StateMachineTask(taskId00, taskId00Partitions, true); expectRestoreToBeCompleted(consumer, changeLogReader); 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 4d7e9d0fb385f..8ea2b9d41edc6 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 @@ -45,15 +45,13 @@ public class LegacySubscriptionInfoSerde { private final Set prevTasks; private final Set standbyTasks; private final String userEndPoint; - private final Map taskLags; public LegacySubscriptionInfoSerde(final int version, final int latestSupportedVersion, final UUID processId, final Set prevTasks, final Set standbyTasks, - final String userEndPoint, - final Map taskLags) { + final String userEndPoint) { if (latestSupportedVersion == UNKNOWN && (version < 1 || version > 2)) { throw new IllegalArgumentException( "Only versions 1 and 2 are expected to use an UNKNOWN (-1) latest supported version. " + @@ -67,11 +65,10 @@ public LegacySubscriptionInfoSerde(final int version, usedVersion = version; this.latestSupportedVersion = latestSupportedVersion; this.processId = processId; - this.prevTasks = prevTasks == null ? taskLagMapToPrevTaskSet(taskLags) : prevTasks; - this.standbyTasks = standbyTasks == null ? taskLagMapToStandbyTaskSet(taskLags) : standbyTasks; + this.prevTasks = prevTasks; + this.standbyTasks = standbyTasks; // Coerce empty string to null. This was the effect of the serialization logic, anyway. this.userEndPoint = userEndPoint == null || userEndPoint.isEmpty() ? null : userEndPoint; - this.taskLags = taskLags == null ? prevAndStandbyTasksToTaskLagMap(prevTasks, standbyTasks) : taskLags; } public int version() { @@ -185,15 +182,6 @@ public static void encodeUserEndPoint(final ByteBuffer buf, } } - public static void encodeTaskLags(final ByteBuffer buf, - final Map taskLags) { - buf.putInt(taskLags.size()); - for (final Map.Entry taskLagEntry : taskLags.entrySet()) { - taskLagEntry.getKey().writeTo(buf); - buf.putInt(taskLagEntry.getValue()); - } - } - public static byte[] prepareUserEndPoint(final String userEndPoint) { if (userEndPoint == null) { return new byte[0]; @@ -217,22 +205,22 @@ public static LegacySubscriptionInfoSerde decode(final ByteBuffer data) { final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); final String userEndPoint = decodeUserEndpoint(data); - return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, processId, prevTasks, standbyTasks, userEndPoint, null); + return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, processId, prevTasks, standbyTasks, userEndPoint); } else if (usedVersion == 2) { final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); final String userEndPoint = decodeUserEndpoint(data); - return new LegacySubscriptionInfoSerde(2, UNKNOWN, processId, prevTasks, standbyTasks, userEndPoint, null); + return new LegacySubscriptionInfoSerde(2, UNKNOWN, processId, prevTasks, standbyTasks, userEndPoint); } else if (usedVersion == 1) { final UUID processId = decodeProcessId(data); final Set prevTasks = decodeTasks(data); final Set standbyTasks = decodeTasks(data); - return new LegacySubscriptionInfoSerde(1, UNKNOWN, processId, prevTasks, standbyTasks, null, null); + return new LegacySubscriptionInfoSerde(1, UNKNOWN, processId, prevTasks, standbyTasks, null); } else { final int latestSupportedVersion = data.getInt(); log.info("Unable to decode subscription data: used version: {}; latest supported version: {}", usedVersion, LATEST_SUPPORTED_VERSION); - return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, null, null, null, null, null); + return new LegacySubscriptionInfoSerde(usedVersion, latestSupportedVersion, null, null, null, null); } } @@ -252,23 +240,6 @@ private static Set decodeTasks(final ByteBuffer data) { return prevTasks; } - private static Set taskLagMapToPrevTaskSet(final Map taskLags) { - return taskLags.entrySet().stream().filter(t -> t.getValue() == 0).map(Map.Entry::getKey).collect(Collectors.toSet()); - } - - private static Set taskLagMapToStandbyTaskSet(final Map taskLags) { - return taskLags.entrySet().stream().filter(t -> t.getValue() != 0).map(Map.Entry::getKey).collect(Collectors.toSet()); - } - - private static Map prevAndStandbyTasksToTaskLagMap(final Set prevTasks, - final Set standbyTasks) { - final Map taskLags = new HashMap<>(prevTasks.stream() - .collect(Collectors.toMap(t -> t, l -> -1))); - taskLags.putAll(standbyTasks.stream() - .collect(Collectors.toMap(t -> t, l -> 0))); - return taskLags; - } - private static UUID decodeProcessId(final ByteBuffer data) { return new UUID(data.getLong(), data.getLong()); } 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 c53b304b88cb9..52dfd0ae723a0 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 @@ -29,6 +29,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -43,20 +44,13 @@ public class SubscriptionInfoTest { private static final Set STANDBY_TASKS = new HashSet<>(Arrays.asList( new TaskId(1, 1), new TaskId(2, 0))); - private static final Map TASK_OFFSET_SUMS = mkMap( - mkEntry(new TaskId(0, 0), -1), - mkEntry(new TaskId(0, 1), -1), - mkEntry(new TaskId(1, 0), -1), - mkEntry(new TaskId(1, 1), 0), - mkEntry(new TaskId(2, 0), 10) + private static final Map TASK_OFFSET_SUMS = mkMap( + mkEntry(new TaskId(0, 0), ACTIVE_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(0, 1), ACTIVE_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(1, 0), ACTIVE_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(1, 1), 0L), + mkEntry(new TaskId(2, 0), 10L) ); - static { - TASK_OFFSET_SUMS.put(new TaskId(0, 0), -1); - TASK_OFFSET_SUMS.put(new TaskId(0, 1), -1); - TASK_OFFSET_SUMS.put(new TaskId(1, 0), -1); - TASK_OFFSET_SUMS.put(new TaskId(1, 1), 0); - TASK_OFFSET_SUMS.put(new TaskId(2, 0), 10); - } private final static String IGNORED_USER_ENDPOINT = "ignoredUserEndpoint:80"; @@ -128,8 +122,7 @@ public void generatedVersion1ShouldDecodeLegacyFormat() { processId, ACTIVE_TASKS, STANDBY_TASKS, - "localhost:80", - null + "localhost:80" ); final ByteBuffer buffer = info.encode(); buffer.rewind(); @@ -188,8 +181,7 @@ public void generatedVersion2ShouldDecodeLegacyFormat() { processId, ACTIVE_TASKS, STANDBY_TASKS, - "localhost:80", - null + "localhost:80" ); final ByteBuffer buffer = info.encode(); buffer.rewind(); @@ -253,8 +245,7 @@ public void generatedVersion3To6ShouldDecodeLegacyFormat() { processId, ACTIVE_TASKS, STANDBY_TASKS, - "localhost:80", - null + "localhost:80" ); final ByteBuffer buffer = info.encode(); buffer.rewind(); 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 0aa9832c3dd08..ad995fad75780 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 @@ -158,8 +158,9 @@ public ByteBuffer subscriptionUserData(final Set topics) { return new FutureSubscriptionInfo( usedSubscriptionMetadataVersion, taskManager.processId(), - userEndPoint(), - taskManager.getTaskOffsetSums()) + SubscriptionInfo.getActiveTasksFromTaskOffsetSumMap(taskManager.getTaskOffsetSums()), + SubscriptionInfo.getStandbyTasksFromTaskOffsetSumMap(taskManager.getTaskOffsetSums()), + userEndPoint()) .encode(); } } @@ -248,7 +249,8 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, info.processId(), - info.userEndPoint(), taskManager().getTaskOffsetSums()) + info.userEndPoint(), + taskManager().getTaskOffsetSums()) .encode(), subscription.ownedPartitions() )); @@ -279,45 +281,44 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr private static class FutureSubscriptionInfo { private final int version; private final UUID processId; + private final Set activeTasks; + private final Set standbyTasks; private final String userEndPoint; - private final Map taskLags; // for testing only; don't apply version checks FutureSubscriptionInfo(final int version, final UUID processId, - final String userEndPoint, - final Map taskLags) { + final Set activeTasks, + final Set standbyTasks, + final String userEndPoint) { this.version = version; this.processId = processId; + this.activeTasks = activeTasks; + this.standbyTasks = standbyTasks; this.userEndPoint = userEndPoint; - this.taskLags = taskLags; if (version <= LATEST_SUPPORTED_VERSION) { throw new IllegalArgumentException("this class can't be used with version " + version); } } private ByteBuffer encode() { - final ByteBuffer buf = encodeFutureVersion(); - buf.rewind(); - return buf; - } - - private ByteBuffer encodeFutureVersion() { final byte[] endPointBytes = LegacySubscriptionInfoSerde.prepareUserEndPoint(userEndPoint); final ByteBuffer buf = ByteBuffer.allocate( 4 + // used version - 4 + // latest supported version version - 16 + // client ID - 4 + endPointBytes.length + // length + endpoint - 4 + taskLags.size() * 12 // length + task lag info + 4 + // latest supported version version + 16 + // client ID + 4 + activeTasks.size() * 8 + // length + active tasks + 4 + standbyTasks.size() * 8 + // length + standby tasks + 4 + endPointBytes.length // length + endpoint ); buf.putInt(version); // used version buf.putInt(version); // supported version LegacySubscriptionInfoSerde.encodeClientUUID(buf, processId); + LegacySubscriptionInfoSerde.encodeTasks(buf, activeTasks); + LegacySubscriptionInfoSerde.encodeTasks(buf, standbyTasks); LegacySubscriptionInfoSerde.encodeUserEndPoint(buf, endPointBytes); - LegacySubscriptionInfoSerde.encodeTaskLags(buf, taskLags); buf.rewind(); From 951c245be7fbbe3a43c96b7f4f21956593ef0613 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 4 Mar 2020 14:22:07 -0500 Subject: [PATCH 17/24] checkstyle --- .../internals/StreamsPartitionAssignor.java | 2 +- .../internals/StreamsPartitionAssignorTest.java | 13 +++++++------ .../assignment/LegacySubscriptionInfoSerde.java | 3 --- 3 files changed, 8 insertions(+), 10 deletions(-) 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 60739891a9a43..f84bb3d4a67bd 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 @@ -565,7 +565,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // there are two cases where we need to construct the prevTasks from the ownedPartitions: // 1) COOPERATIVE clients on version 2.4-2.5 do not encode active tasks and rely on ownedPartitions instead // 2) future clientduring version probing: we can't decode the future subscription info's prev tasks - if (!state.ownedPartitions().isEmpty() && (uuid == futureId ||state.prevActiveTasks().isEmpty())) { + if (!state.ownedPartitions().isEmpty() && (uuid == futureId || state.prevActiveTasks().isEmpty())) { final Set previousActiveTasks = new HashSet<>(); for (final Map.Entry partitionEntry : state.ownedPartitions().entrySet()) { final TopicPartition tp = partitionEntry.getKey(); 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 96059031b7709..9ece80b09d389 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 @@ -72,6 +72,7 @@ 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.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -232,10 +233,10 @@ private static SubscriptionInfo getInfo(final int version, } private static SubscriptionInfo getInfo(final UUID processId, - final Map taskLags, + final Map taskOffsetSums, final String userEndPoint) { return new SubscriptionInfo( - LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, userEndPoint, taskLags); + LATEST_SUPPORTED_VERSION, LATEST_SUPPORTED_VERSION, processId, userEndPoint, taskOffsetSums); } private static SubscriptionInfo getInfo(final UUID processId, final Set prevTasks, final Set standbyTasks) { @@ -1982,10 +1983,10 @@ private static void assertEquivalentAssignment(final Map> t } } - static Map getTaskOffsetSums(final Set activeTasks, final Set standbyTasks) { - final Map taskLags = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> -1)); - taskLags.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0))); - return taskLags; + static Map getTaskOffsetSums(final Set activeTasks, final Set standbyTasks) { + final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> ACTIVE_TASK_SENTINEL_OFFSET)); + taskOffsetSums.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0L))); + return taskOffsetSums; } } 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 8ea2b9d41edc6..e8976ee53ea48 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 @@ -18,9 +18,6 @@ import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; -import java.util.HashMap; -import java.util.Map; -import java.util.stream.Collectors; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.processor.TaskId; import org.slf4j.Logger; From 41d70a24b09f6c500ab0db780ec723a5b5003c1f Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 4 Mar 2020 15:35:34 -0500 Subject: [PATCH 18/24] only encode RUNNING active tasks as -1 --- .../streams/processor/internals/TaskManager.java | 11 ++++++----- .../internals/assignment/SubscriptionInfo.java | 6 +++--- .../internals/StreamsPartitionAssignorTest.java | 4 ++-- .../internals/assignment/SubscriptionInfoTest.java | 8 ++++---- 4 files changed, 15 insertions(+), 14 deletions(-) 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 25d32cdb5677a..943d083be3670 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 @@ -51,7 +51,8 @@ import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; -import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; +import static org.apache.kafka.streams.processor.internals.Task.State.RUNNING; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; public class TaskManager { // initialize the task list @@ -362,8 +363,8 @@ public Map getTaskOffsetSums() { final Map taskOffsetSums = new HashMap<>(); for (final TaskId id : tasksOnLocalStorage()) { - if (isActive(id)) { - taskOffsetSums.put(id, ACTIVE_TASK_SENTINEL_OFFSET); + if (isRunning(id)) { + taskOffsetSums.put(id, RUNNING_TASK_SENTINEL_OFFSET); } else { taskOffsetSums.put(id, 0L); } @@ -489,9 +490,9 @@ private Stream standbyTaskStream() { return tasks.values().stream().filter(t -> !t.isActive()); } - private boolean isActive(final TaskId id) { + private boolean isRunning(final TaskId id) { final Task task = tasks.get(id); - return task != null && task.isActive(); + return task != null && task.isActive() && task.state() == RUNNING; } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 38b84c772dfa6..98ebbb2f05843 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -40,7 +40,7 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); // encode running active tasks as -1 to skip computing their offset sum since we know they are caught up - public static final long ACTIVE_TASK_SENTINEL_OFFSET = -1; + public static final long RUNNING_TASK_SENTINEL_OFFSET = -1; static final int UNKNOWN = -1; private final SubscriptionInfoData data; @@ -115,7 +115,7 @@ private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Subscr final Set standbyTasks = new HashSet<>(); for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { - if (taskOffsetSum.getValue() == ACTIVE_TASK_SENTINEL_OFFSET) { + if (taskOffsetSum.getValue() == RUNNING_TASK_SENTINEL_OFFSET) { prevTasks.add(taskOffsetSum.getKey()); } else { standbyTasks.add(taskOffsetSum.getKey()); @@ -213,7 +213,7 @@ public static Set getStandbyTasksFromTaskOffsetSumMap(final Map taskOffsetSumMapToTaskSet(final Map taskOffsetSums, final boolean getActiveTasks) { return taskOffsetSums.entrySet().stream() - .filter(t -> getActiveTasks == (t.getValue() == ACTIVE_TASK_SENTINEL_OFFSET)) + .filter(t -> getActiveTasks == (t.getValue() == RUNNING_TASK_SENTINEL_OFFSET)) .map(Map.Entry::getKey) .collect(Collectors.toSet()); } 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 9ece80b09d389..1206bde2958ce 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 @@ -72,7 +72,7 @@ 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.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -1984,7 +1984,7 @@ private static void assertEquivalentAssignment(final Map> t } static Map getTaskOffsetSums(final Set activeTasks, final Set standbyTasks) { - final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> ACTIVE_TASK_SENTINEL_OFFSET)); + final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> RUNNING_TASK_SENTINEL_OFFSET)); taskOffsetSums.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0L))); return taskOffsetSums; } 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 52dfd0ae723a0..5d7451155a021 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 @@ -29,7 +29,7 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; -import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.ACTIVE_TASK_SENTINEL_OFFSET; +import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -45,9 +45,9 @@ public class SubscriptionInfoTest { new TaskId(1, 1), new TaskId(2, 0))); private static final Map TASK_OFFSET_SUMS = mkMap( - mkEntry(new TaskId(0, 0), ACTIVE_TASK_SENTINEL_OFFSET), - mkEntry(new TaskId(0, 1), ACTIVE_TASK_SENTINEL_OFFSET), - mkEntry(new TaskId(1, 0), ACTIVE_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(0, 0), RUNNING_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(0, 1), RUNNING_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(1, 0), RUNNING_TASK_SENTINEL_OFFSET), mkEntry(new TaskId(1, 1), 0L), mkEntry(new TaskId(2, 0), 10L) ); From aeebfa8b60f7d99c36cd0b3854ae3700bd25f5d3 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 4 Mar 2020 15:39:00 -0500 Subject: [PATCH 19/24] fix javadocs --- .../apache/kafka/streams/processor/internals/TaskManager.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 943d083be3670..6c398f3ea289d 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 @@ -357,7 +357,7 @@ void handleLostAll() { } /** - * @return Map from task id to that task's overall lag across all state stores + * @return Map from task id to its total offset summed across all state stores */ public Map getTaskOffsetSums() { final Map taskOffsetSums = new HashMap<>(); From 16be04627aec19705a439a75b20a2a68ea08ded6 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Wed, 4 Mar 2020 16:49:46 -0500 Subject: [PATCH 20/24] reuse existing Task.LATEST_OFFSET for active running task sentinel offset --- .../kafka/streams/processor/internals/TaskManager.java | 3 +-- .../processor/internals/assignment/SubscriptionInfo.java | 7 +++---- 2 files changed, 4 insertions(+), 6 deletions(-) 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 6c398f3ea289d..691a06ac8275c 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 @@ -52,7 +52,6 @@ import static org.apache.kafka.streams.processor.internals.Task.State.CREATED; import static org.apache.kafka.streams.processor.internals.Task.State.RESTORING; import static org.apache.kafka.streams.processor.internals.Task.State.RUNNING; -import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; public class TaskManager { // initialize the task list @@ -364,7 +363,7 @@ public Map getTaskOffsetSums() { for (final TaskId id : tasksOnLocalStorage()) { if (isRunning(id)) { - taskOffsetSums.put(id, RUNNING_TASK_SENTINEL_OFFSET); + taskOffsetSums.put(id, Task.LATEST_OFFSET); } else { taskOffsetSums.put(id, 0L); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 98ebbb2f05843..73af3bd269b7e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.internals.generated.SubscriptionInfoData; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.TaskOffsetSum; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.Task; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -39,8 +40,6 @@ public class SubscriptionInfo { private static final Logger LOG = LoggerFactory.getLogger(SubscriptionInfo.class); - // encode running active tasks as -1 to skip computing their offset sum since we know they are caught up - public static final long RUNNING_TASK_SENTINEL_OFFSET = -1; static final int UNKNOWN = -1; private final SubscriptionInfoData data; @@ -115,7 +114,7 @@ private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Subscr final Set standbyTasks = new HashSet<>(); for (final Map.Entry taskOffsetSum : taskOffsetSums.entrySet()) { - if (taskOffsetSum.getValue() == RUNNING_TASK_SENTINEL_OFFSET) { + if (taskOffsetSum.getValue() == Task.LATEST_OFFSET) { prevTasks.add(taskOffsetSum.getKey()); } else { standbyTasks.add(taskOffsetSum.getKey()); @@ -213,7 +212,7 @@ public static Set getStandbyTasksFromTaskOffsetSumMap(final Map taskOffsetSumMapToTaskSet(final Map taskOffsetSums, final boolean getActiveTasks) { return taskOffsetSums.entrySet().stream() - .filter(t -> getActiveTasks == (t.getValue() == RUNNING_TASK_SENTINEL_OFFSET)) + .filter(t -> getActiveTasks == (t.getValue() == Task.LATEST_OFFSET)) .map(Map.Entry::getKey) .collect(Collectors.toSet()); } From cce68476ac15ff67bc5db4a49945170f73da076f Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Thu, 5 Mar 2020 14:24:49 -0800 Subject: [PATCH 21/24] github review --- .../streams/processor/internals/Task.java | 2 +- .../assignment/SubscriptionInfo.java | 52 +++++++++---------- .../assignment/SubscriptionInfoTest.java | 10 ++-- 3 files changed, 31 insertions(+), 33 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java index 18d12bffb5b7b..332c39e1d52c3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Task.java @@ -32,7 +32,7 @@ import java.util.Set; public interface Task { - + // this must be negative to distinguish a running active task from other kinds tasks which may be caught up to the same offsets long LATEST_OFFSET = -2L; /* diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 73af3bd269b7e..7e7bc812bfa54 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -90,15 +90,19 @@ public SubscriptionInfo(final int version, data.setLatestSupportedVersion(latestSupportedVersion); } if (version >= 7) { - setTaskOffsetSumDataFromTaskOffsetSumMap(data, taskOffsetSums); + setTaskOffsetSumDataFromTaskOffsetSumMap(taskOffsetSums); } else { - setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums); + setPrevAndStandbySetsFromParsedTaskOffsetSumMap(taskOffsetSums); } this.data = data; } - private static void setTaskOffsetSumDataFromTaskOffsetSumMap(final SubscriptionInfoData data, - final Map taskOffsetSums) { + private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) { + validateVersions(subscriptionInfoData.version(), subscriptionInfoData.latestSupportedVersion()); + this.data = subscriptionInfoData; + } + + private void setTaskOffsetSumDataFromTaskOffsetSumMap(final Map taskOffsetSums) { data.setTaskOffsetSums(taskOffsetSums.entrySet().stream().map(t -> { final SubscriptionInfoData.TaskOffsetSum taskOffsetSum = new SubscriptionInfoData.TaskOffsetSum(); taskOffsetSum.setTopicGroupId(t.getKey().topicGroupId); @@ -108,8 +112,7 @@ private static void setTaskOffsetSumDataFromTaskOffsetSumMap(final SubscriptionI }).collect(Collectors.toList())); } - private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final SubscriptionInfoData data, - final Map taskOffsetSums) { + private void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Map taskOffsetSums) { final Set prevTasks = new HashSet<>(); final Set standbyTasks = new HashSet<>(); @@ -135,11 +138,6 @@ private static void setPrevAndStandbySetsFromParsedTaskOffsetSumMap(final Subscr }).collect(Collectors.toList())); } - private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) { - validateVersions(subscriptionInfoData.version(), subscriptionInfoData.latestSupportedVersion()); - this.data = subscriptionInfoData; - } - public int version() { return data.version(); } @@ -154,32 +152,32 @@ public UUID processId() { public Set prevTasks() { if (prevTasksCache == null) { - // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { - setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums()); + prevTasksCache = getActiveTasksFromTaskOffsetSumMap(taskOffsetSums()); + } else { + prevTasksCache = Collections.unmodifiableSet( + data.prevTasks() + .stream() + .map(t -> new TaskId(t.topicGroupId(), t.partition())) + .collect(Collectors.toSet()) + ); } - prevTasksCache = Collections.unmodifiableSet( - data.prevTasks() - .stream() - .map(t -> new TaskId(t.topicGroupId(), t.partition())) - .collect(Collectors.toSet()) - ); } return prevTasksCache; } public Set standbyTasks() { if (standbyTasksCache == null) { - // lazily initialize the prev and standby task maps as they may not be needed if (data.version() >= 7) { - setPrevAndStandbySetsFromParsedTaskOffsetSumMap(data, taskOffsetSums()); + standbyTasksCache = getStandbyTasksFromTaskOffsetSumMap(taskOffsetSums()); + } else { + standbyTasksCache = Collections.unmodifiableSet( + data.standbyTasks() + .stream() + .map(t -> new TaskId(t.topicGroupId(), t.partition())) + .collect(Collectors.toSet()) + ); } - standbyTasksCache = Collections.unmodifiableSet( - data.standbyTasks() - .stream() - .map(t -> new TaskId(t.topicGroupId(), t.partition())) - .collect(Collectors.toSet()) - ); } return standbyTasksCache; } 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 5d7451155a021..cf79a5f301e45 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 @@ -18,6 +18,7 @@ import java.util.Map; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.streams.processor.internals.Task; import org.junit.Test; import java.nio.ByteBuffer; @@ -29,7 +30,6 @@ import static org.apache.kafka.common.utils.Utils.mkEntry; import static org.apache.kafka.common.utils.Utils.mkMap; import static org.apache.kafka.streams.processor.internals.assignment.StreamsAssignmentProtocolVersions.LATEST_SUPPORTED_VERSION; -import static org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -45,9 +45,9 @@ public class SubscriptionInfoTest { new TaskId(1, 1), new TaskId(2, 0))); private static final Map TASK_OFFSET_SUMS = mkMap( - mkEntry(new TaskId(0, 0), RUNNING_TASK_SENTINEL_OFFSET), - mkEntry(new TaskId(0, 1), RUNNING_TASK_SENTINEL_OFFSET), - mkEntry(new TaskId(1, 0), RUNNING_TASK_SENTINEL_OFFSET), + mkEntry(new TaskId(0, 0), Task.LATEST_OFFSET), + mkEntry(new TaskId(0, 1), Task.LATEST_OFFSET), + mkEntry(new TaskId(1, 0), Task.LATEST_OFFSET), mkEntry(new TaskId(1, 1), 0L), mkEntry(new TaskId(2, 0), 10L) ); @@ -303,7 +303,7 @@ public void shouldConvertTaskOffsetSumMapToTaskSetsForOlderVersion() { private static ByteBuffer encodeFutureVersion() { final ByteBuffer buf = ByteBuffer.allocate(4 /* used version */ - + 4 /* supported version */); + + 4 /* supported version */); buf.putInt(LATEST_SUPPORTED_VERSION + 1); buf.putInt(LATEST_SUPPORTED_VERSION + 1); buf.rewind(); From 36b94cc2d87783c943f3189965ad25fb88ae7748 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Thu, 5 Mar 2020 15:03:25 -0800 Subject: [PATCH 22/24] encode task id offset mapcompactly --- .../internals/StreamsPartitionAssignor.java | 2 +- .../assignment/SubscriptionInfo.java | 33 ++++++++++++++----- .../common/message/SubscriptionInfo.json | 11 +++++++ .../StreamsPartitionAssignorTest.java | 3 +- 4 files changed, 37 insertions(+), 12 deletions(-) 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 f84bb3d4a67bd..f035a2c8174df 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 @@ -564,7 +564,7 @@ public GroupAssignment assign(final Cluster metadata, final GroupSubscription gr // there are two cases where we need to construct the prevTasks from the ownedPartitions: // 1) COOPERATIVE clients on version 2.4-2.5 do not encode active tasks and rely on ownedPartitions instead - // 2) future clientduring version probing: we can't decode the future subscription info's prev tasks + // 2) future client during version probing, when we can't decode the future subscription info's prev tasks if (!state.ownedPartitions().isEmpty() && (uuid == futureId || state.prevActiveTasks().isEmpty())) { final Set previousActiveTasks = new HashSet<>(); for (final Map.Entry partitionEntry : state.ownedPartitions().entrySet()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 7e7bc812bfa54..95b9cc406ea0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -16,12 +16,16 @@ */ package org.apache.kafka.streams.processor.internals.assignment; +import java.util.ArrayList; +import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; import org.apache.kafka.common.protocol.ByteBufferAccessor; import org.apache.kafka.common.protocol.ObjectSerializationCache; import org.apache.kafka.streams.errors.TaskAssignmentException; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData; +import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.PartitionToOffsetSum; import org.apache.kafka.streams.internals.generated.SubscriptionInfoData.TaskOffsetSum; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.Task; @@ -103,11 +107,20 @@ private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) { } private void setTaskOffsetSumDataFromTaskOffsetSumMap(final Map taskOffsetSums) { - data.setTaskOffsetSums(taskOffsetSums.entrySet().stream().map(t -> { + final Map> topicGroupIdToPartitionOffsetSum = new HashMap<>(); + for (final Map.Entry taskEntry : taskOffsetSums.entrySet()) { + final TaskId task = taskEntry.getKey(); + topicGroupIdToPartitionOffsetSum.putIfAbsent(task.topicGroupId, new ArrayList<>()); + topicGroupIdToPartitionOffsetSum.get(task.topicGroupId).add( + new SubscriptionInfoData.PartitionToOffsetSum() + .setPartition(task.partition) + .setOffsetSum(taskEntry.getValue())); + } + + data.setTaskOffsetSums(topicGroupIdToPartitionOffsetSum.entrySet().stream().map(t -> { final SubscriptionInfoData.TaskOffsetSum taskOffsetSum = new SubscriptionInfoData.TaskOffsetSum(); - taskOffsetSum.setTopicGroupId(t.getKey().topicGroupId); - taskOffsetSum.setPartition(t.getKey().partition); - taskOffsetSum.setOffsetSum(t.getValue()); + taskOffsetSum.setTopicGroupId(t.getKey()); + taskOffsetSum.setPartitionToOffsetSum(t.getValue()); return taskOffsetSum; }).collect(Collectors.toList())); } @@ -184,11 +197,13 @@ public Set standbyTasks() { public Map taskOffsetSums() { if (taskOffsetSumsCache == null) { - taskOffsetSumsCache = Collections.unmodifiableMap( - data.taskOffsetSums() - .stream() - .collect(Collectors.toMap(t -> new TaskId(t.topicGroupId(), t.partition()), TaskOffsetSum::offsetSum)) - ); + taskOffsetSumsCache = new HashMap<>(); + for (final TaskOffsetSum topicGroup : data.taskOffsetSums()) { + for (final PartitionToOffsetSum partitionOffsetSum : topicGroup.partitionToOffsetSum()) { + taskOffsetSumsCache.put(new TaskId(topicGroup.topicGroupId(), partitionOffsetSum.partition()), + partitionOffsetSum.offsetSum()); + } + } } return taskOffsetSumsCache; } diff --git a/streams/src/main/resources/common/message/SubscriptionInfo.json b/streams/src/main/resources/common/message/SubscriptionInfo.json index 21716363bb065..acd3b6e21c6eb 100644 --- a/streams/src/main/resources/common/message/SubscriptionInfo.json +++ b/streams/src/main/resources/common/message/SubscriptionInfo.json @@ -80,6 +80,17 @@ "versions": "7+", "type": "int32" }, + { + "name": "partitionToOffsetSum", + "versions": "7+", + "type": "[]PartitionToOffsetSum" + } + ] + }, + { + "name": "PartitionToOffsetSum", + "versions": "7+", + "fields": [ { "name": "partition", "versions": "7+", 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 1206bde2958ce..3dba29a622860 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 @@ -72,7 +72,6 @@ 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.SubscriptionInfo.RUNNING_TASK_SENTINEL_OFFSET; import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.not; import static org.hamcrest.MatcherAssert.assertThat; @@ -1984,7 +1983,7 @@ private static void assertEquivalentAssignment(final Map> t } static Map getTaskOffsetSums(final Set activeTasks, final Set standbyTasks) { - final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> RUNNING_TASK_SENTINEL_OFFSET)); + final Map taskOffsetSums = activeTasks.stream().collect(Collectors.toMap(t -> t, t -> Task.LATEST_OFFSET)); taskOffsetSums.putAll(standbyTasks.stream().collect(Collectors.toMap(t -> t, t -> 0L))); return taskOffsetSums; } From f0f20224659759d920218cf9be2ac91a6dc3d6ae Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Thu, 5 Mar 2020 15:09:54 -0800 Subject: [PATCH 23/24] fx NPE --- .../processor/internals/assignment/SubscriptionInfo.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index 95b9cc406ea0e..f4af02036ea7f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -93,12 +93,14 @@ public SubscriptionInfo(final int version, if (version >= 3) { data.setLatestSupportedVersion(latestSupportedVersion); } + + this.data = data; + if (version >= 7) { setTaskOffsetSumDataFromTaskOffsetSumMap(taskOffsetSums); } else { setPrevAndStandbySetsFromParsedTaskOffsetSumMap(taskOffsetSums); } - this.data = data; } private SubscriptionInfo(final SubscriptionInfoData subscriptionInfoData) { From 9c1849c0c1506c3db22a16d0954faf76135b89e2 Mon Sep 17 00:00:00 2001 From: ableegoldman Date: Thu, 5 Mar 2020 15:38:21 -0800 Subject: [PATCH 24/24] fix VP upgrade test --- .../processor/internals/StreamsPartitionAssignor.java | 6 +++++- .../processor/internals/assignment/SubscriptionInfo.java | 2 +- .../org/apache/kafka/streams/tests/StreamsUpgradeTest.java | 3 +-- 3 files changed, 7 insertions(+), 4 deletions(-) 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 f035a2c8174df..639509fad72d2 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 @@ -219,7 +219,7 @@ public ByteBuffer subscriptionUserData(final Set topics) { // 1. Client UUID (a unique id assigned to an instance of KafkaStreams) // 2. Map from task id to its overall lag - taskManager.handleRebalanceStart(topics); + handleRebalanceStart(topics); return new SubscriptionInfo( usedSubscriptionMetadataVersion, @@ -1269,4 +1269,8 @@ protected TaskManager taskManager() { return taskManager; } + protected void handleRebalanceStart(final Set topics) { + taskManager.handleRebalanceStart(topics); + } + } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java index f4af02036ea7f..e718b1c0595cb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java @@ -95,7 +95,7 @@ public SubscriptionInfo(final int version, } this.data = data; - + if (version >= 7) { setTaskOffsetSumDataFromTaskOffsetSumMap(taskOffsetSums); } else { 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 ad995fad75780..8862008e710d3 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 @@ -138,13 +138,12 @@ public void configure(final Map configs) { @Override public ByteBuffer subscriptionUserData(final Set topics) { - super.subscriptionUserData(topics); - // Adds the following information to subscription // 1. Client UUID (a unique id assigned to an instance of KafkaStreams) // 2. Task ids of previously running tasks // 3. Task ids of valid local states on the client's state directory. final TaskManager taskManager = taskManager(); + handleRebalanceStart(topics); if (usedSubscriptionMetadataVersion <= LATEST_SUPPORTED_VERSION) { return new SubscriptionInfo(