lucasbru commented on code in PR #18527: URL: https://github.com/apache/kafka/pull/18527#discussion_r1922435303
########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); Review Comment: The collections should be unmodifiable, and probably private with getters? Right now, I can do `Assignment.EMPTY.activeTasks.add(new TaskID("a",1))` and wreak havoc. If you want to make the class mutable, you cannot define `EMPTY` as a static variable. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskId taskId = (TaskId) o; + return partitionId == taskId.partitionId && Objects.equals(subtopologyId, taskId.subtopologyId); + } + + @Override + public int hashCode() { + return Objects.hash(subtopologyId, partitionId); + } + + @Override + public int compareTo(TaskId taskId) { + if (subtopologyId.equals(taskId.subtopologyId)) { Review Comment: In Java 8, it woul be more idiomatic to use soemthing like: ``` Comparator.comparing(TaskId::subtopologyId) .thenComparingInt(TaskId::partitionId); ``` ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); Review Comment: Should we require the parameters to be non-null? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskId taskId = (TaskId) o; + return partitionId == taskId.partitionId && Objects.equals(subtopologyId, taskId.subtopologyId); + } + + @Override + public int hashCode() { + return Objects.hash(subtopologyId, partitionId); + } + + @Override + public int compareTo(TaskId taskId) { + if (subtopologyId.equals(taskId.subtopologyId)) { + return partitionId - taskId.partitionId; + } + return subtopologyId.compareTo(taskId.subtopologyId); + } + + @Override + public String toString() { + return "TaskId{" + + "subtopologyId=" + subtopologyId + + ", partitionId=" + partitionId + + '}'; + } + } + + public static class Subtopology { + + public final Set<String> sourceTopics; Review Comment: We may want to make these collections unmodifiable, non-null, private and have getters. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { Review Comment: If you are calling this constructor, you should just be using `EMPTY`. It does not make sense to have both. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; Review Comment: Non-Null? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskId taskId = (TaskId) o; + return partitionId == taskId.partitionId && Objects.equals(subtopologyId, taskId.subtopologyId); + } + + @Override + public int hashCode() { + return Objects.hash(subtopologyId, partitionId); + } + + @Override + public int compareTo(TaskId taskId) { + if (subtopologyId.equals(taskId.subtopologyId)) { + return partitionId - taskId.partitionId; + } + return subtopologyId.compareTo(taskId.subtopologyId); + } + + @Override + public String toString() { + return "TaskId{" + + "subtopologyId=" + subtopologyId + + ", partitionId=" + partitionId + + '}'; + } + } + + public static class Subtopology { + + public final Set<String> sourceTopics; + public final Set<String> repartitionSinkTopics; + public final Map<String, TopicInfo> stateChangelogTopics; + public final Map<String, TopicInfo> repartitionSourceTopics; + public final Collection<Set<String>> copartitionGroups; + + public Subtopology(final Set<String> sourceTopics, + final Set<String> repartitionSinkTopics, + final Map<String, TopicInfo> repartitionSourceTopics, + final Map<String, TopicInfo> stateChangelogTopics, + final Collection<Set<String>> copartitionGroups + ) { + this.sourceTopics = sourceTopics; + this.repartitionSinkTopics = repartitionSinkTopics; + this.stateChangelogTopics = stateChangelogTopics; + this.repartitionSourceTopics = repartitionSourceTopics; + this.copartitionGroups = copartitionGroups; + } + + @Override + public String toString() { + return "Subtopology{" + + "sourceTopics=" + sourceTopics + + ", repartitionSinkTopics=" + repartitionSinkTopics + + ", stateChangelogTopics=" + stateChangelogTopics + + ", repartitionSourceTopics=" + repartitionSourceTopics + + ", copartitionGroups=" + copartitionGroups + + '}'; + } + } + + public static class TopicInfo { + + public final Optional<Integer> numPartitions; + public final Optional<Short> replicationFactor; + public final Map<String, String> topicConfigs; + + public TopicInfo(final Optional<Integer> numPartitions, + final Optional<Short> replicationFactor, + final Map<String, String> topicConfigs) { + this.numPartitions = numPartitions; + this.replicationFactor = replicationFactor; + this.topicConfigs = topicConfigs; + } + + @Override + public String toString() { + return "TopicInfo{" + + "numPartitions=" + numPartitions + + ", replicationFactor=" + replicationFactor + + ", topicConfigs=" + topicConfigs + + '}'; + } + } + + private final Map<String, Subtopology> subtopologies; Review Comment: nit: Ideally, move fields towards the top of the class. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskId taskId = (TaskId) o; + return partitionId == taskId.partitionId && Objects.equals(subtopologyId, taskId.subtopologyId); + } + + @Override + public int hashCode() { + return Objects.hash(subtopologyId, partitionId); + } + + @Override + public int compareTo(TaskId taskId) { + if (subtopologyId.equals(taskId.subtopologyId)) { + return partitionId - taskId.partitionId; + } + return subtopologyId.compareTo(taskId.subtopologyId); + } + + @Override + public String toString() { + return "TaskId{" + + "subtopologyId=" + subtopologyId + + ", partitionId=" + partitionId + + '}'; + } + } + + public static class Subtopology { + + public final Set<String> sourceTopics; + public final Set<String> repartitionSinkTopics; + public final Map<String, TopicInfo> stateChangelogTopics; + public final Map<String, TopicInfo> repartitionSourceTopics; + public final Collection<Set<String>> copartitionGroups; + + public Subtopology(final Set<String> sourceTopics, + final Set<String> repartitionSinkTopics, + final Map<String, TopicInfo> repartitionSourceTopics, + final Map<String, TopicInfo> stateChangelogTopics, + final Collection<Set<String>> copartitionGroups + ) { + this.sourceTopics = sourceTopics; + this.repartitionSinkTopics = repartitionSinkTopics; + this.stateChangelogTopics = stateChangelogTopics; + this.repartitionSourceTopics = repartitionSourceTopics; + this.copartitionGroups = copartitionGroups; + } + + @Override + public String toString() { + return "Subtopology{" + + "sourceTopics=" + sourceTopics + + ", repartitionSinkTopics=" + repartitionSinkTopics + + ", stateChangelogTopics=" + stateChangelogTopics + + ", repartitionSourceTopics=" + repartitionSourceTopics + + ", copartitionGroups=" + copartitionGroups + + '}'; + } + } + + public static class TopicInfo { + + public final Optional<Integer> numPartitions; + public final Optional<Short> replicationFactor; + public final Map<String, String> topicConfigs; + + public TopicInfo(final Optional<Integer> numPartitions, + final Optional<Short> replicationFactor, + final Map<String, String> topicConfigs) { + this.numPartitions = numPartitions; + this.replicationFactor = replicationFactor; + this.topicConfigs = topicConfigs; + } + + @Override + public String toString() { + return "TopicInfo{" + + "numPartitions=" + numPartitions + + ", replicationFactor=" + replicationFactor + + ", topicConfigs=" + topicConfigs + + '}'; + } + } + + private final Map<String, Subtopology> subtopologies; + + private final AtomicReference<Assignment> reconciledAssignment = new AtomicReference<>(Assignment.EMPTY); + + public StreamsRebalanceData(Map<String, Subtopology> subtopologies) { + this.subtopologies = subtopologies; Review Comment: non-null, unmodifiable? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceEventsProcessor.java: ########## @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnAllTasksLostCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnAllTasksLostCallbackNeededEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksAssignedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksAssignedCallbackNeededEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksRevokedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksRevokedCallbackNeededEvent; +import org.apache.kafka.common.KafkaException; + +import java.util.LinkedList; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * Processes events from the Streams rebalance protocol. + * <p> + * The Streams rebalance processor receives events from the background thread of the async consumer, more precisely + * from the Streams membership manager and handles them. + * For example, events are requests for invoking the task assignment and task revocation callbacks. + * Results of the event handling are passed back to the background thread. + */ +public class StreamsRebalanceEventsProcessor { + + private final BlockingQueue<BackgroundEvent> onCallbackRequests = new LinkedBlockingQueue<>(); + private ApplicationEventHandler applicationEventHandler = null; + private final StreamsGroupRebalanceCallbacks rebalanceCallbacks; + private final StreamsRebalanceData streamsRebalanceData; + + /** + * Constructs the Streams rebalance processor. + * + * @param streamsRebalanceData + * @param rebalanceCallbacks + */ + public StreamsRebalanceEventsProcessor(StreamsRebalanceData streamsRebalanceData, + StreamsGroupRebalanceCallbacks rebalanceCallbacks) { + this.streamsRebalanceData = streamsRebalanceData; + this.rebalanceCallbacks = rebalanceCallbacks; + } + + /** + * Requests the invocation of the task assignment callback. + * + * @param assignment The tasks to be assigned to the member of the Streams group. + * @return A future that will be completed when the callback has been invoked. + */ + public CompletableFuture<Void> requestOnTasksAssignedCallbackInvocation(final StreamsRebalanceData.Assignment assignment) { + final StreamsOnTasksAssignedCallbackNeededEvent onTasksAssignedCallbackNeededEvent = new StreamsOnTasksAssignedCallbackNeededEvent(assignment); + onCallbackRequests.add(onTasksAssignedCallbackNeededEvent); + return onTasksAssignedCallbackNeededEvent.future(); + } + + /** + * Requests the invocation of the task revocation callback. + * + * @param activeTasksToRevoke The tasks to revoke from the member of the Streams group + * @return A future that will be completed when the callback has been invoked. + */ + public CompletableFuture<Void> requestOnTasksRevokedCallbackInvocation(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke) { + final StreamsOnTasksRevokedCallbackNeededEvent onTasksRevokedCallbackNeededEvent = new StreamsOnTasksRevokedCallbackNeededEvent(activeTasksToRevoke); + onCallbackRequests.add(onTasksRevokedCallbackNeededEvent); + return onTasksRevokedCallbackNeededEvent.future(); + } + + /** + * Requests the invocation of the all tasks lost callback. + * + * @return A future that will be completed when the callback has been invoked. + */ + public CompletableFuture<Void> requestOnAllTasksLostCallbackInvocation() { + final StreamsOnAllTasksLostCallbackNeededEvent onAllTasksLostCallbackNeededEvent = new StreamsOnAllTasksLostCallbackNeededEvent(); + onCallbackRequests.add(onAllTasksLostCallbackNeededEvent); + return onAllTasksLostCallbackNeededEvent.future(); + } + + /** + * Sets the application event handler. + * + * The application handler sends the results of the callbacks to the background thread. + * + * @param applicationEventHandler The application handler. + */ + public void setApplicationEventHandler(final ApplicationEventHandler applicationEventHandler) { + this.applicationEventHandler = applicationEventHandler; + } + + private void process(final BackgroundEvent event) { + switch (event.type()) { + case ERROR: + throw ((ErrorEvent) event).error(); + + case STREAMS_ON_TASKS_REVOKED_CALLBACK_NEEDED: + processStreamsOnTasksRevokedCallbackNeededEvent((StreamsOnTasksRevokedCallbackNeededEvent) event); + break; + + case STREAMS_ON_TASKS_ASSIGNED_CALLBACK_NEEDED: + processStreamsOnTasksAssignedCallbackNeededEvent((StreamsOnTasksAssignedCallbackNeededEvent) event); + break; + + case STREAMS_ON_ALL_TASKS_LOST_CALLBACK_NEEDED: + processStreamsOnAllTasksLostCallbackNeededEvent((StreamsOnAllTasksLostCallbackNeededEvent) event); + break; + + default: + throw new IllegalArgumentException("Background event type " + event.type() + " was not expected"); + + } + } + + private void processStreamsOnTasksRevokedCallbackNeededEvent(final StreamsOnTasksRevokedCallbackNeededEvent event) { + StreamsOnTasksRevokedCallbackCompletedEvent invokedEvent = invokeOnTasksRevokedCallback(event.activeTasksToRevoke(), event.future()); + applicationEventHandler.add(invokedEvent); + if (invokedEvent.error().isPresent()) { + throw invokedEvent.error().get(); + } + } + + private void processStreamsOnTasksAssignedCallbackNeededEvent(final StreamsOnTasksAssignedCallbackNeededEvent event) { + StreamsOnTasksAssignedCallbackCompletedEvent invokedEvent = invokeOnTasksAssignedCallback(event.assignment(), event.future()); + applicationEventHandler.add(invokedEvent); + if (invokedEvent.error().isPresent()) { + throw invokedEvent.error().get(); + } + } + + private void processStreamsOnAllTasksLostCallbackNeededEvent(final StreamsOnAllTasksLostCallbackNeededEvent event) { + StreamsOnAllTasksLostCallbackCompletedEvent invokedEvent = invokeOnAllTasksLostCallback(event.future()); + applicationEventHandler.add(invokedEvent); + if (invokedEvent.error().isPresent()) { + throw invokedEvent.error().get(); + } + } + + private StreamsOnTasksRevokedCallbackCompletedEvent invokeOnTasksRevokedCallback(final Set<StreamsRebalanceData.TaskId> activeTasksToRevoke, + final CompletableFuture<Void> future) { + final Optional<Exception> exceptionFromCallback = rebalanceCallbacks.onTasksRevoked(activeTasksToRevoke); Review Comment: `invokeOnTasksRevokedCallback` has a different implementation than `invokeOnTasksAssignedCallback` and `invokeOnAllTasksLostCallback`. It's probably easier to read if all three use either the `.map` stile or the `isPresent` style. I don't mind which one, `isPresent` is probably a little easier to follow. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/events/BackgroundEvent.java: ########## @@ -27,7 +27,9 @@ public abstract class BackgroundEvent { public enum Type { - ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK + ERROR, CONSUMER_REBALANCE_LISTENER_CALLBACK_NEEDED, SHARE_ACKNOWLEDGEMENT_COMMIT_CALLBACK, Review Comment: nit: one per line will simplify conflict resolution ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceEventsProcessor.java: ########## @@ -0,0 +1,205 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import org.apache.kafka.clients.consumer.internals.events.ApplicationEventHandler; +import org.apache.kafka.clients.consumer.internals.events.BackgroundEvent; +import org.apache.kafka.clients.consumer.internals.events.ErrorEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnAllTasksLostCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnAllTasksLostCallbackNeededEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksAssignedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksAssignedCallbackNeededEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksRevokedCallbackCompletedEvent; +import org.apache.kafka.clients.consumer.internals.events.StreamsOnTasksRevokedCallbackNeededEvent; +import org.apache.kafka.common.KafkaException; + +import java.util.LinkedList; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.LinkedBlockingQueue; + +/** + * Processes events from the Streams rebalance protocol. + * <p> + * The Streams rebalance processor receives events from the background thread of the async consumer, more precisely + * from the Streams membership manager and handles them. + * For example, events are requests for invoking the task assignment and task revocation callbacks. + * Results of the event handling are passed back to the background thread. + */ +public class StreamsRebalanceEventsProcessor { + + private final BlockingQueue<BackgroundEvent> onCallbackRequests = new LinkedBlockingQueue<>(); + private ApplicationEventHandler applicationEventHandler = null; + private final StreamsGroupRebalanceCallbacks rebalanceCallbacks; + private final StreamsRebalanceData streamsRebalanceData; + + /** + * Constructs the Streams rebalance processor. + * + * @param streamsRebalanceData + * @param rebalanceCallbacks + */ + public StreamsRebalanceEventsProcessor(StreamsRebalanceData streamsRebalanceData, + StreamsGroupRebalanceCallbacks rebalanceCallbacks) { + this.streamsRebalanceData = streamsRebalanceData; Review Comment: non-null? ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { + this.subtopologyId = subtopologyId; + this.partitionId = partitionId; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + TaskId taskId = (TaskId) o; + return partitionId == taskId.partitionId && Objects.equals(subtopologyId, taskId.subtopologyId); + } + + @Override + public int hashCode() { + return Objects.hash(subtopologyId, partitionId); + } + + @Override + public int compareTo(TaskId taskId) { + if (subtopologyId.equals(taskId.subtopologyId)) { + return partitionId - taskId.partitionId; + } + return subtopologyId.compareTo(taskId.subtopologyId); + } + + @Override + public String toString() { + return "TaskId{" + + "subtopologyId=" + subtopologyId + + ", partitionId=" + partitionId + + '}'; + } + } + + public static class Subtopology { + + public final Set<String> sourceTopics; + public final Set<String> repartitionSinkTopics; + public final Map<String, TopicInfo> stateChangelogTopics; + public final Map<String, TopicInfo> repartitionSourceTopics; + public final Collection<Set<String>> copartitionGroups; + + public Subtopology(final Set<String> sourceTopics, + final Set<String> repartitionSinkTopics, + final Map<String, TopicInfo> repartitionSourceTopics, + final Map<String, TopicInfo> stateChangelogTopics, + final Collection<Set<String>> copartitionGroups + ) { + this.sourceTopics = sourceTopics; + this.repartitionSinkTopics = repartitionSinkTopics; + this.stateChangelogTopics = stateChangelogTopics; + this.repartitionSourceTopics = repartitionSourceTopics; + this.copartitionGroups = copartitionGroups; + } + + @Override + public String toString() { + return "Subtopology{" + + "sourceTopics=" + sourceTopics + + ", repartitionSinkTopics=" + repartitionSinkTopics + + ", stateChangelogTopics=" + stateChangelogTopics + + ", repartitionSourceTopics=" + repartitionSourceTopics + + ", copartitionGroups=" + copartitionGroups + + '}'; + } + } + + public static class TopicInfo { + + public final Optional<Integer> numPartitions; Review Comment: We may want to make these fields unmodifiable, non-null, private and have getters. ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { + + public static class Assignment { + + public static final Assignment EMPTY = new Assignment(); + + public final Set<TaskId> activeTasks = new HashSet<>(); + + public final Set<TaskId> standbyTasks = new HashSet<>(); + + public final Set<TaskId> warmupTasks = new HashSet<>(); + + public Assignment() { + } + + public Assignment(final Set<TaskId> activeTasks, + final Set<TaskId> standbyTasks, + final Set<TaskId> warmupTasks) { + this.activeTasks.addAll(activeTasks); + this.standbyTasks.addAll(standbyTasks); + this.warmupTasks.addAll(warmupTasks); + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final Assignment that = (Assignment) o; + return Objects.equals(activeTasks, that.activeTasks) + && Objects.equals(standbyTasks, that.standbyTasks) + && Objects.equals(warmupTasks, that.warmupTasks); + } + + @Override + public int hashCode() { + return Objects.hash(activeTasks, standbyTasks, warmupTasks); + } + + public Assignment copy() { + return new Assignment(activeTasks, standbyTasks, warmupTasks); + } + + @Override + public String toString() { + return "Assignment{" + + "activeTasks=" + activeTasks + + ", standbyTasks=" + standbyTasks + + ", warmupTasks=" + warmupTasks + + '}'; + } + } + + public static class TaskId implements Comparable<TaskId> { + + private final String subtopologyId; + private final int partitionId; + + public int partitionId() { + return partitionId; + } + + public String subtopologyId() { + return subtopologyId; + } + + public TaskId(final String subtopologyId, final int partitionId) { Review Comment: nit: first constructor, then getters ########## clients/src/main/java/org/apache/kafka/clients/consumer/internals/StreamsRebalanceData.java: ########## @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.clients.consumer.internals; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicReference; + +/** + * This class holds the data that is needed to participate in the Streams rebalance protocol. + */ +public class StreamsRebalanceData { Review Comment: This class and its inner classes need unit tests. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
