diff --git a/checkstyle/import-control-coordinator-common.xml b/checkstyle/import-control-coordinator-common.xml
index bafffe8069746..60c37807568a1 100644
--- a/checkstyle/import-control-coordinator-common.xml
+++ b/checkstyle/import-control-coordinator-common.xml
@@ -58,6 +58,7 @@
+
diff --git a/checkstyle/import-control-jmh-benchmarks.xml b/checkstyle/import-control-jmh-benchmarks.xml
index 6e78ea01853e0..d2f87a3577f4f 100644
--- a/checkstyle/import-control-jmh-benchmarks.xml
+++ b/checkstyle/import-control-jmh-benchmarks.xml
@@ -52,6 +52,8 @@
+
+
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataDelta.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataDelta.java
new file mode 100644
index 0000000000000..c1fa7f35cc251
--- /dev/null
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataDelta.java
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.Collection;
+import java.util.Set;
+
+/**
+ * Provides metadata deltas to Coordinators (GroupCoordinator, ShareCoordinator, etc) such as changed topics and deleted topics
+ * Implementations should be immutable.
+ */
+public interface CoordinatorMetadataDelta {
+
+ CoordinatorMetadataDelta EMPTY = emptyDelta();
+
+ Collection createdTopicIds();
+
+ Collection changedTopicIds();
+
+ Set deletedTopicIds();
+
+ /**
+ * Returns the previous image of the coordinator metadata.
+ * This image is a snapshot of the metadata before the delta occurred.
+ */
+ CoordinatorMetadataImage image();
+
+ private static CoordinatorMetadataDelta emptyDelta() {
+ return new CoordinatorMetadataDelta() {
+ @Override
+ public Collection createdTopicIds() {
+ return Set.of();
+ }
+
+ @Override
+ public Collection changedTopicIds() {
+ return Set.of();
+ }
+
+ @Override
+ public Set deletedTopicIds() {
+ return Set.of();
+ }
+
+ @Override
+ public CoordinatorMetadataImage image() {
+ return CoordinatorMetadataImage.EMPTY;
+ }
+ };
+ }
+}
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataImage.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataImage.java
new file mode 100644
index 0000000000000..d294bddf51bae
--- /dev/null
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorMetadataImage.java
@@ -0,0 +1,99 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * Provides metadata to Coordinators (GroupCoordinator, ShareCoordinator, etc) such as topics, partitions, and their configurations.
+ * Implementations should be thread-safe and immutable.
+ */
+public interface CoordinatorMetadataImage {
+ CoordinatorMetadataImage EMPTY = emptyImage();
+
+ Set topicIds();
+
+ Set topicNames();
+
+ Optional topicMetadata(String topicName);
+
+ Optional topicMetadata(Uuid topicId);
+
+ CoordinatorMetadataDelta emptyDelta();
+
+ long version();
+
+ boolean isEmpty();
+
+ /**
+ * Metadata about a particular topic
+ */
+ interface TopicMetadata {
+ String name();
+
+ Uuid id();
+
+ int partitionCount();
+
+ List partitionRacks(int partitionId);
+ }
+
+ private static CoordinatorMetadataImage emptyImage() {
+
+ return new CoordinatorMetadataImage() {
+ @Override
+ public Set topicIds() {
+ return Set.of();
+ }
+
+ @Override
+ public Set topicNames() {
+ return Set.of();
+ }
+
+ @Override
+ public Optional topicMetadata(String topicName) {
+ return Optional.empty();
+ }
+
+ @Override
+ public Optional topicMetadata(Uuid topicId) {
+ return Optional.empty();
+ }
+
+ @Override
+ public CoordinatorMetadataDelta emptyDelta() {
+ return CoordinatorMetadataDelta.EMPTY;
+ }
+
+ @Override
+ public long version() {
+ return 0L;
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return true;
+ }
+ };
+ }
+
+}
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
index 8f72b0cac50ba..27dd6518f1aa0 100644
--- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntime.java
@@ -39,8 +39,6 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.deferred.DeferredEvent;
import org.apache.kafka.deferred.DeferredEventQueue;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.util.timer.Timer;
import org.apache.kafka.server.util.timer.TimerTask;
import org.apache.kafka.storage.internals.log.LogConfig;
@@ -2016,7 +2014,7 @@ public void onHighWatermarkUpdated(
/**
* The latest known metadata image.
*/
- private volatile MetadataImage metadataImage = MetadataImage.EMPTY;
+ private volatile CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* Constructor.
@@ -2481,18 +2479,18 @@ public void scheduleUnloadOperation(
* @param delta The metadata delta.
*/
public void onNewMetadataImage(
- MetadataImage newImage,
- MetadataDelta delta
+ CoordinatorMetadataImage newImage,
+ CoordinatorMetadataDelta delta
) {
throwIfNotRunning();
- log.debug("Scheduling applying of a new metadata image with offset {}.", newImage.offset());
+ log.debug("Scheduling applying of a new metadata image with version {}.", newImage.version());
// Update global image.
metadataImage = newImage;
// Push an event for each coordinator.
coordinators.keySet().forEach(tp -> {
- scheduleInternalOperation("UpdateImage(tp=" + tp + ", offset=" + newImage.offset() + ")", tp, () -> {
+ scheduleInternalOperation("UpdateImage(tp=" + tp + ", version=" + newImage.version() + ")", tp, () -> {
CoordinatorContext context = coordinators.get(tp);
if (context != null) {
context.lock.lock();
@@ -2500,18 +2498,18 @@ public void onNewMetadataImage(
if (context.state == CoordinatorState.ACTIVE) {
// The new image can be applied to the coordinator only if the coordinator
// exists and is in the active state.
- log.debug("Applying new metadata image with offset {} to {}.", newImage.offset(), tp);
+ log.debug("Applying new metadata image with version {} to {}.", newImage.version(), tp);
context.coordinator.onNewMetadataImage(newImage, delta);
} else {
- log.debug("Ignored new metadata image with offset {} for {} because the coordinator is not active.",
- newImage.offset(), tp);
+ log.debug("Ignored new metadata image with version {} for {} because the coordinator is not active.",
+ newImage.version(), tp);
}
} finally {
context.lock.unlock();
}
} else {
- log.debug("Ignored new metadata image with offset {} for {} because the coordinator does not exist.",
- newImage.offset(), tp);
+ log.debug("Ignored new metadata image with version {} for {} because the coordinator does not exist.",
+ newImage.version(), tp);
}
});
});
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorShard.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorShard.java
index 6b0f40ddf3323..7734b12751525 100644
--- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorShard.java
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/CoordinatorShard.java
@@ -17,8 +17,6 @@
package org.apache.kafka.coordinator.common.runtime;
import org.apache.kafka.common.requests.TransactionResult;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
/**
* CoordinatorShard is basically a replicated state machine managed by the
@@ -32,16 +30,16 @@ public interface CoordinatorShard {
*
* @param newImage The metadata image.
*/
- default void onLoaded(MetadataImage newImage) {}
+ default void onLoaded(CoordinatorMetadataImage newImage) {}
/**
- * A new metadata image is available. This is only called after {@link CoordinatorShard#onLoaded(MetadataImage)}
+ * A new metadata image is available. This is only called after {@link CoordinatorShard#onLoaded(CoordinatorMetadataImage)}
* is called to signal that the coordinator has been fully loaded.
*
* @param newImage The new metadata image.
* @param delta The delta image.
*/
- default void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {}
+ default void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {}
/**
* The coordinator has been unloaded. This is used to apply
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDelta.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDelta.java
new file mode 100644
index 0000000000000..8e340d81c887b
--- /dev/null
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDelta.java
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.image.MetadataDelta;
+
+import java.util.Collection;
+import java.util.Set;
+
+/**
+ * An implementation of {@link CoordinatorMetadataDelta} that wraps the KRaft MetadataDelta.
+ */
+public class KRaftCoordinatorMetadataDelta implements CoordinatorMetadataDelta {
+
+ final MetadataDelta metadataDelta;
+
+ public KRaftCoordinatorMetadataDelta(MetadataDelta metadataDelta) {
+ this.metadataDelta = metadataDelta;
+ }
+
+ @Override
+ public Collection createdTopicIds() {
+ if (metadataDelta == null || metadataDelta.topicsDelta() == null) {
+ return Set.of();
+ }
+ return metadataDelta.topicsDelta().createdTopicIds();
+ }
+
+ @Override
+ public Collection changedTopicIds() {
+ if (metadataDelta == null || metadataDelta.topicsDelta() == null) {
+ return Set.of();
+ }
+ return metadataDelta.topicsDelta().changedTopics().keySet();
+ }
+
+ @Override
+ public Set deletedTopicIds() {
+ if (metadataDelta == null || metadataDelta.topicsDelta() == null) {
+ return Set.of();
+ }
+ return metadataDelta.topicsDelta().deletedTopicIds();
+ }
+
+ @Override
+ public CoordinatorMetadataImage image() {
+ return new KRaftCoordinatorMetadataImage(metadataDelta.image());
+ }
+
+ @Override
+ public String toString() {
+ return metadataDelta.toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !o.getClass().equals(this.getClass())) return false;
+ KRaftCoordinatorMetadataDelta other = (KRaftCoordinatorMetadataDelta) o;
+ return metadataDelta.equals(other.metadataDelta);
+ }
+
+ @Override
+ public int hashCode() {
+ return metadataDelta.hashCode();
+ }
+}
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImage.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImage.java
new file mode 100644
index 0000000000000..c0284a4aed6b4
--- /dev/null
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImage.java
@@ -0,0 +1,149 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.image.ClusterImage;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.TopicImage;
+import org.apache.kafka.metadata.BrokerRegistration;
+import org.apache.kafka.metadata.PartitionRegistration;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.Set;
+
+/**
+ * An implementation of {@link CoordinatorMetadataImage} that wraps the KRaft MetadataImage.
+ */
+public class KRaftCoordinatorMetadataImage implements CoordinatorMetadataImage {
+
+ private final MetadataImage metadataImage;
+
+ public KRaftCoordinatorMetadataImage(MetadataImage metadataImage) {
+ this.metadataImage = metadataImage;
+ }
+
+ @Override
+ public Set topicIds() {
+ return Collections.unmodifiableSet(metadataImage.topics().topicsById().keySet());
+ }
+
+ @Override
+ public Set topicNames() {
+ return Collections.unmodifiableSet(metadataImage.topics().topicsByName().keySet());
+ }
+
+ @Override
+ public Optional topicMetadata(Uuid topicId) {
+ TopicImage topicImage = metadataImage.topics().getTopic(topicId);
+ if (topicImage == null) return Optional.empty();
+
+ ClusterImage clusterImage = metadataImage.cluster();
+ if (clusterImage == null) return Optional.empty();
+
+ return Optional.of(new KraftTopicMetadata(topicImage, clusterImage));
+ }
+
+ @Override
+ public Optional topicMetadata(String topicName) {
+ TopicImage topicImage = metadataImage.topics().getTopic(topicName);
+ if (topicImage == null) return Optional.empty();
+
+ ClusterImage clusterImage = metadataImage.cluster();
+ if (clusterImage == null) return Optional.empty();
+
+ return Optional.of(new KraftTopicMetadata(topicImage, clusterImage));
+ }
+
+ @Override
+ public CoordinatorMetadataDelta emptyDelta() {
+ return new KRaftCoordinatorMetadataDelta(new MetadataDelta(metadataImage));
+ }
+
+ @Override
+ public long version() {
+ return metadataImage.offset();
+ }
+
+ @Override
+ public boolean isEmpty() {
+ return metadataImage.isEmpty();
+ }
+
+ @Override
+ public String toString() {
+ return metadataImage.toString();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (o == null || !o.getClass().equals(this.getClass())) return false;
+ KRaftCoordinatorMetadataImage other = (KRaftCoordinatorMetadataImage) o;
+ return metadataImage.equals(other.metadataImage);
+ }
+
+ @Override
+ public int hashCode() {
+ return metadataImage.hashCode();
+ }
+
+ public static class KraftTopicMetadata implements TopicMetadata {
+ private final TopicImage topicImage;
+ private final ClusterImage clusterImage;
+
+ public KraftTopicMetadata(TopicImage topicImage, ClusterImage clusterImage) {
+ this.topicImage = topicImage;
+ this.clusterImage = clusterImage;
+ }
+
+ @Override
+ public String name() {
+ return topicImage.name();
+ }
+
+ @Override
+ public Uuid id() {
+ return topicImage.id();
+ }
+
+ @Override
+ public int partitionCount() {
+ return topicImage.partitions().size();
+ }
+
+ @Override
+ public List partitionRacks(int partition) {
+ List racks = new ArrayList<>();
+ PartitionRegistration partitionRegistration = topicImage.partitions().get(partition);
+ if (partitionRegistration != null) {
+ for (int replicaId : partitionRegistration.replicas) {
+ BrokerRegistration broker = clusterImage.broker(replicaId);
+ if (broker != null) {
+ broker.rack().ifPresent(racks::add);
+ }
+ }
+ return racks;
+ } else {
+ return List.of();
+ }
+ }
+ }
+}
diff --git a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java
index 3aa622cc98b08..4dc2506d9434d 100644
--- a/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java
+++ b/coordinator-common/src/main/java/org/apache/kafka/coordinator/common/runtime/SnapshottableCoordinator.java
@@ -19,7 +19,6 @@
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.requests.TransactionResult;
import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.timeline.SnapshotRegistry;
@@ -179,7 +178,7 @@ public synchronized void updateLastCommittedOffset(Long offset) {
*
* @param newImage The metadata image.
*/
- synchronized void onLoaded(MetadataImage newImage) {
+ synchronized void onLoaded(CoordinatorMetadataImage newImage) {
this.coordinator.onLoaded(newImage);
}
@@ -207,7 +206,7 @@ synchronized long lastWrittenOffset() {
* @param newImage The new metadata image.
* @param delta The delta image.
*/
- synchronized void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
+ synchronized void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {
this.coordinator.onNewMetadataImage(newImage, delta);
}
diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
index f0d38d749fa2c..4a040df6712c0 100644
--- a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/CoordinatorRuntimeTest.java
@@ -32,7 +32,6 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.MetadataProvenance;
import org.apache.kafka.server.util.FutureUtils;
import org.apache.kafka.server.util.timer.MockTimer;
import org.apache.kafka.storage.internals.log.LogConfig;
@@ -155,7 +154,7 @@ public void testScheduleLoading() {
assertEquals(ACTIVE, ctx.state);
// Verify that onLoaded is called.
- verify(coordinator, times(1)).onLoaded(MetadataImage.EMPTY);
+ verify(coordinator, times(1)).onLoaded(CoordinatorMetadataImage.EMPTY);
// Verify that the listener is registered.
verify(writer, times(1)).registerListener(
@@ -1897,11 +1896,11 @@ public void testOnNewMetadataImage() {
// Coordinator 0 is loaded. It should get the current image
// that is the empty one.
future0.complete(null);
- verify(coordinator0).onLoaded(MetadataImage.EMPTY);
+ verify(coordinator0).onLoaded(CoordinatorMetadataImage.EMPTY);
// Publish a new image.
- MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
- MetadataImage newImage = delta.apply(MetadataProvenance.EMPTY);
+ CoordinatorMetadataDelta delta = new KRaftCoordinatorMetadataDelta(new MetadataDelta(MetadataImage.EMPTY));
+ CoordinatorMetadataImage newImage = CoordinatorMetadataImage.EMPTY;
runtime.onNewMetadataImage(newImage, delta);
// Coordinator 0 should be notified about it.
diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDeltaTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDeltaTest.java
new file mode 100644
index 0000000000000..f65103e87d858
--- /dev/null
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataDeltaTest.java
@@ -0,0 +1,132 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.common.metadata.PartitionChangeRecord;
+import org.apache.kafka.common.metadata.RemoveTopicRecord;
+import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.image.MetadataDelta;
+import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.MetadataProvenance;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.Collection;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+public class KRaftCoordinatorMetadataDeltaTest {
+
+ @Test
+ public void testKRaftCoordinatorDeltaWithNulls() {
+ assertTrue(new KRaftCoordinatorMetadataDelta(null).changedTopicIds().isEmpty());
+ assertTrue(new KRaftCoordinatorMetadataDelta(new MetadataDelta(MetadataImage.EMPTY)).changedTopicIds().isEmpty());
+
+ assertTrue(new KRaftCoordinatorMetadataDelta(null).deletedTopicIds().isEmpty());
+ assertTrue(new KRaftCoordinatorMetadataDelta(new MetadataDelta(MetadataImage.EMPTY)).deletedTopicIds().isEmpty());
+
+ assertTrue(new KRaftCoordinatorMetadataDelta(null).createdTopicIds().isEmpty());
+ assertTrue(new KRaftCoordinatorMetadataDelta(new MetadataDelta(MetadataImage.EMPTY)).createdTopicIds().isEmpty());
+ }
+
+ @Test
+ public void testKRaftCoordinatorDelta() {
+ Uuid topicId = Uuid.randomUuid();
+ String topicName = "test-topic";
+ Uuid topicId2 = Uuid.randomUuid();
+ String topicName2 = "test-topic2";
+ Uuid deletedTopicId = Uuid.randomUuid();
+ String deletedTopicName = "deleted-topic";
+ Uuid changedTopicId = Uuid.randomUuid();
+ String changedTopicName = "changed-topic";
+
+ MetadataImage image = new MetadataImageBuilder()
+ .addTopic(deletedTopicId, deletedTopicName, 1)
+ .addTopic(changedTopicId, changedTopicName, 1)
+ .build();
+ MetadataDelta delta = new MetadataDelta(image);
+ delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName));
+ delta.replay(new TopicRecord().setTopicId(topicId2).setName(topicName2));
+ delta.replay(new RemoveTopicRecord().setTopicId(deletedTopicId));
+ delta.replay(new PartitionChangeRecord().setTopicId(changedTopicId).setPartitionId(0));
+
+ KRaftCoordinatorMetadataDelta coordinatorDelta = new KRaftCoordinatorMetadataDelta(delta);
+
+ // created topics
+ Collection createdTopicIds = coordinatorDelta.createdTopicIds();
+ assertNotNull(createdTopicIds);
+ assertEquals(2, createdTopicIds.size());
+ assertTrue(createdTopicIds.contains(topicId));
+ assertTrue(createdTopicIds.contains(topicId2));
+
+ // deleted topics
+ Set deletedTopicIds = coordinatorDelta.deletedTopicIds();
+ assertNotNull(deletedTopicIds);
+ assertEquals(1, deletedTopicIds.size());
+ assertTrue(deletedTopicIds.contains(deletedTopicId));
+
+ // changed topics (also includes created topics)
+ Collection changedTopicIds = coordinatorDelta.changedTopicIds();
+ assertNotNull(changedTopicIds);
+ assertEquals(3, changedTopicIds.size());
+ assertTrue(changedTopicIds.contains(changedTopicId));
+ assertTrue(changedTopicIds.contains(topicId));
+ assertTrue(changedTopicIds.contains(topicId2));
+
+ CoordinatorMetadataImage coordinatorImage = coordinatorDelta.image();
+ // the image only contains the original topics, not the new topics yet since we never called delta.apply()
+ assertNotNull(coordinatorImage);
+ assertEquals(Set.of(deletedTopicName, changedTopicName), coordinatorImage.topicNames());
+
+ // the image contains the correct topics after calling apply
+ MetadataImage imageAfterApply = delta.apply(new MetadataProvenance(123, 0, 0L, true));
+ CoordinatorMetadataImage coordinatorImageApply = new KRaftCoordinatorMetadataImage(imageAfterApply);
+ assertNotNull(coordinatorImageApply);
+ assertEquals(Set.of(topicName, topicName2, changedTopicName), coordinatorImageApply.topicNames());
+ }
+
+ @Test
+ public void testEqualsAndHashcode() {
+ Uuid topicId = Uuid.randomUuid();
+ String topicName = "test-topic";
+ Uuid topicId2 = Uuid.randomUuid();
+ String topicName2 = "test-topic2";
+ Uuid topicId3 = Uuid.randomUuid();
+ String topicName3 = "test-topic3";
+
+ MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
+ delta.replay(new TopicRecord().setTopicId(topicId).setName(topicName));
+ delta.replay(new TopicRecord().setTopicId(topicId2).setName(topicName2));
+
+ KRaftCoordinatorMetadataDelta coordinatorDelta = new KRaftCoordinatorMetadataDelta(delta);
+ KRaftCoordinatorMetadataDelta coordinatorDeltaCopy = new KRaftCoordinatorMetadataDelta(delta);
+
+ MetadataDelta delta2 = new MetadataDelta(MetadataImage.EMPTY);
+ delta.replay(new TopicRecord().setTopicId(topicId3).setName(topicName3));
+ KRaftCoordinatorMetadataDelta coordinatorDelta2 = new KRaftCoordinatorMetadataDelta(delta2);
+
+ assertEquals(coordinatorDelta, coordinatorDeltaCopy);
+ assertEquals(coordinatorDelta.hashCode(), coordinatorDeltaCopy.hashCode());
+ assertNotEquals(coordinatorDelta, coordinatorDelta2);
+ assertNotEquals(coordinatorDelta.hashCode(), coordinatorDelta2.hashCode());
+ }
+}
diff --git a/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImageTest.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImageTest.java
new file mode 100644
index 0000000000000..88975713d8ac2
--- /dev/null
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/KRaftCoordinatorMetadataImageTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.coordinator.common.runtime;
+
+import org.apache.kafka.common.Uuid;
+import org.apache.kafka.image.MetadataImage;
+
+import org.junit.jupiter.api.Test;
+
+import java.util.List;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.fail;
+
+class KRaftCoordinatorMetadataImageTest {
+
+ @Test
+ public void testKRaftCoordinatorMetadataImage() {
+ Uuid topicId = Uuid.randomUuid();
+ String topicName = "test-topic";
+ int partitionCount = 2;
+ Uuid topicId2 = Uuid.randomUuid();
+ String topicName2 = "test-topic2";
+ int partitionCount2 = 4;
+ Uuid noPartitionTopicId = Uuid.randomUuid();
+ String noPartitionTopic = "no-partition-topic";
+ long imageVersion = 123L;
+
+ MetadataImage metadataImage = new MetadataImageBuilder()
+ .addTopic(topicId, topicName, partitionCount)
+ .addTopic(topicId2, topicName2, partitionCount2)
+ .addTopic(noPartitionTopicId, noPartitionTopic, 0)
+ .addRacks()
+ .build(imageVersion);
+
+ KRaftCoordinatorMetadataImage image = new KRaftCoordinatorMetadataImage(metadataImage);
+
+ assertEquals(Set.of(topicName, topicName2, noPartitionTopic), image.topicNames());
+ assertEquals(Set.of(topicId, topicId2, noPartitionTopicId), image.topicIds());
+
+ image.topicMetadata(topicName).ifPresentOrElse(
+ topicMetadata -> {
+ assertEquals(topicName, topicMetadata.name());
+ assertEquals(topicId, topicMetadata.id());
+ assertEquals(partitionCount, topicMetadata.partitionCount());
+ List racks0 = topicMetadata.partitionRacks(0);
+ List racks1 = topicMetadata.partitionRacks(1);
+ assertEquals(2, racks0.size());
+ assertEquals(2, racks1.size());
+ assertEquals("rack0", racks0.get(0));
+ assertEquals("rack1", racks0.get(1));
+ assertEquals("rack1", racks1.get(0));
+ assertEquals("rack2", racks1.get(1));
+ },
+ () -> fail("Expected topic metadata for " + topicName)
+ );
+
+ image.topicMetadata(topicName2).ifPresentOrElse(
+ topicMetadata -> {
+ assertEquals(topicName2, topicMetadata.name());
+ assertEquals(topicId2, topicMetadata.id());
+ assertEquals(partitionCount2, topicMetadata.partitionCount());
+ List racks0 = topicMetadata.partitionRacks(0);
+ List racks1 = topicMetadata.partitionRacks(1);
+ assertEquals(2, racks0.size());
+ assertEquals(2, racks1.size());
+ assertEquals("rack0", racks0.get(0));
+ assertEquals("rack1", racks0.get(1));
+ assertEquals("rack1", racks1.get(0));
+ assertEquals("rack2", racks1.get(1));
+ },
+ () -> fail("Expected topic metadata for " + topicName)
+ );
+
+ image.topicMetadata(noPartitionTopic).ifPresentOrElse(
+ topicMetadata -> {
+ assertEquals(noPartitionTopic, topicMetadata.name());
+ assertEquals(noPartitionTopicId, topicMetadata.id());
+ assertEquals(0, topicMetadata.partitionCount());
+ List racks = topicMetadata.partitionRacks(0);
+ assertEquals(0, racks.size());
+ },
+ () -> fail("Expected topic metadata for " + topicName)
+ );
+
+ assertNotNull(image.emptyDelta());
+
+ assertEquals(metadataImage.offset(), image.version());
+ assertEquals(imageVersion, image.version());
+
+ assertFalse(image.isEmpty());
+ }
+
+ @Test
+ public void testEqualsAndHashcode() {
+ Uuid topicId = Uuid.randomUuid();
+ String topicName = "test-topic";
+ int partitionCount = 2;
+ Uuid topicId2 = Uuid.randomUuid();
+ String topicName2 = "test-topic2";
+ int partitionCount2 = 4;
+ long imageVersion = 123L;
+
+ MetadataImage metadataImage = new MetadataImageBuilder()
+ .addTopic(topicId, topicName, partitionCount)
+ .addRacks()
+ .build(imageVersion);
+
+ KRaftCoordinatorMetadataImage coordinatorMetadataImage = new KRaftCoordinatorMetadataImage(metadataImage);
+ KRaftCoordinatorMetadataImage coordinatorMetadataImageCopy = new KRaftCoordinatorMetadataImage(metadataImage);
+
+ MetadataImage metadataImage2 = new MetadataImageBuilder()
+ .addTopic(topicId2, topicName2, partitionCount2)
+ .addRacks()
+ .build(imageVersion);
+
+ KRaftCoordinatorMetadataImage coordinatorMetadataImage2 = new KRaftCoordinatorMetadataImage(metadataImage2);
+
+ assertEquals(coordinatorMetadataImage, coordinatorMetadataImageCopy);
+ assertNotEquals(coordinatorMetadataImage, coordinatorMetadataImage2);
+
+ assertEquals(coordinatorMetadataImage.hashCode(), coordinatorMetadataImageCopy.hashCode());
+ assertNotEquals(coordinatorMetadataImage.hashCode(), coordinatorMetadataImage2.hashCode());
+ }
+}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MetadataImageBuilder.java b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/MetadataImageBuilder.java
similarity index 88%
rename from group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MetadataImageBuilder.java
rename to coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/MetadataImageBuilder.java
index 23a01a6024176..142915a69e610 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/MetadataImageBuilder.java
+++ b/coordinator-common/src/test/java/org/apache/kafka/coordinator/common/runtime/MetadataImageBuilder.java
@@ -14,7 +14,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.kafka.coordinator.group;
+package org.apache.kafka.coordinator.common.runtime;
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.PartitionRecord;
@@ -75,4 +75,12 @@ public MetadataImage build() {
public MetadataImage build(long version) {
return delta.apply(new MetadataProvenance(version, 0, 0L, true));
}
+
+ public CoordinatorMetadataImage buildCoordinatorMetadataImage() {
+ return new KRaftCoordinatorMetadataImage(build());
+ }
+
+ public CoordinatorMetadataImage buildCoordinatorMetadataImage(long version) {
+ return new KRaftCoordinatorMetadataImage(build(version));
+ }
}
diff --git a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
index 1b82f2744ea1c..7876163f9ea98 100644
--- a/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
+++ b/core/src/main/scala/kafka/server/metadata/BrokerMetadataPublisher.scala
@@ -26,6 +26,7 @@ import kafka.utils.Logging
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.errors.TimeoutException
import org.apache.kafka.common.internals.Topic
+import org.apache.kafka.coordinator.common.runtime.{KRaftCoordinatorMetadataDelta, KRaftCoordinatorMetadataImage}
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.coordinator.transaction.TransactionLogConfig
@@ -233,7 +234,7 @@ class BrokerMetadataPublisher(
try {
// Propagate the new image to the group coordinator.
- groupCoordinator.onNewMetadataImage(newImage, delta)
+ groupCoordinator.onNewMetadataImage(new KRaftCoordinatorMetadataImage(newImage), new KRaftCoordinatorMetadataDelta(delta))
} catch {
case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating group " +
s"coordinator with local changes in $deltaName", t)
@@ -241,7 +242,7 @@ class BrokerMetadataPublisher(
try {
// Propagate the new image to the share coordinator.
- shareCoordinator.onNewMetadataImage(newImage, delta)
+ shareCoordinator.onNewMetadataImage(new KRaftCoordinatorMetadataImage(newImage), newImage.features(), new KRaftCoordinatorMetadataDelta(delta))
} catch {
case t: Throwable => metadataPublishingFaultHandler.handleFault("Error updating share " +
s"coordinator with local changes in $deltaName", t)
diff --git a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
index 9a7c18ad079e5..3bb3f5fc3f77e 100644
--- a/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
+++ b/core/src/test/scala/unit/kafka/server/metadata/BrokerMetadataPublisherTest.scala
@@ -35,6 +35,7 @@ import org.apache.kafka.common.internals.Topic
import org.apache.kafka.common.metadata.{FeatureLevelRecord, PartitionRecord, RemoveTopicRecord, TopicRecord}
import org.apache.kafka.common.test.{KafkaClusterTestKit, TestKitNodes}
import org.apache.kafka.common.utils.Exit
+import org.apache.kafka.coordinator.common.runtime.{KRaftCoordinatorMetadataDelta, KRaftCoordinatorMetadataImage}
import org.apache.kafka.coordinator.group.GroupCoordinator
import org.apache.kafka.coordinator.share.ShareCoordinator
import org.apache.kafka.image.{AclsImage, ClientQuotasImage, ClusterImageTest, ConfigurationsImage, DelegationTokenImage, FeaturesImage, MetadataDelta, MetadataImage, MetadataImageTest, MetadataProvenance, ProducerIdsImage, ScramImage, TopicsImage}
@@ -291,7 +292,7 @@ class BrokerMetadataPublisherTest {
.numBytes(42)
.build())
- verify(groupCoordinator).onNewMetadataImage(image, delta)
+ verify(groupCoordinator).onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(delta))
}
@Test
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
index 9559c4bb52f31..2b8a1f2715203 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinator.java
@@ -53,9 +53,9 @@
import org.apache.kafka.common.message.TxnOffsetCommitResponseData;
import org.apache.kafka.common.requests.TransactionResult;
import org.apache.kafka.common.utils.BufferSupplier;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import java.time.Duration;
@@ -469,8 +469,8 @@ void onResignation(
* @param delta The metadata delta.
*/
void onNewMetadataImage(
- MetadataImage newImage,
- MetadataDelta delta
+ CoordinatorMetadataImage newImage,
+ CoordinatorMetadataDelta delta
);
/**
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
index fdc23d8d3402b..662b91126e0f4 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java
@@ -84,6 +84,8 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
@@ -94,9 +96,6 @@
import org.apache.kafka.coordinator.group.api.assignor.ConsumerGroupPartitionAssignor;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.record.BrokerCompressionType;
@@ -329,9 +328,9 @@ public GroupCoordinatorService build() {
/**
* The metadata image to extract topic id to names map.
- * This is initialised when the {@link GroupCoordinator#onNewMetadataImage(MetadataImage, MetadataDelta)} is called
+ * This is initialised when the {@link GroupCoordinator#onNewMetadataImage(CoordinatorMetadataImage, CoordinatorMetadataDelta)} is called
*/
- private MetadataImage metadataImage = null;
+ private CoordinatorMetadataImage metadataImage = null;
/**
*
@@ -1689,8 +1688,9 @@ public CompletableFuture readStateSummaryData = new ArrayList<>(requestData.topics().size());
List describeShareGroupOffsetsResponseTopicList = new ArrayList<>(requestData.topics().size());
requestData.topics().forEach(topic -> {
- Uuid topicId = metadataImage.topics().topicNameToIdView().get(topic.topicName());
- if (topicId != null) {
+ Optional topicMetadataOpt = metadataImage.topicMetadata(topic.topicName());
+ if (topicMetadataOpt.isPresent()) {
+ var topicId = topicMetadataOpt.get().id();
requestTopicIdToNameMapping.put(topicId, topic.topicName());
readStateSummaryData.add(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData()
.setTopicId(topicId)
@@ -1757,9 +1757,8 @@ public CompletableFuture {
- String topicName = metadataImage.topics().topicIdToNameView().get(topicId);
- if (topicName != null) {
- requestTopicIdToNameMapping.put(topicId, topicName);
+ metadataImage.topicMetadata(topicId).ifPresent(topicMetadata -> {
+ requestTopicIdToNameMapping.put(topicId, topicMetadata.name());
readSummaryRequestData.topics().add(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData()
.setTopicId(topicId)
.setPartitions(
@@ -1767,7 +1766,7 @@ public CompletableFuture new ReadShareGroupStateSummaryRequestData.PartitionData().setPartition(partitionIndex)
).toList()
));
- }
+ });
});
return readShareGroupStateSummary(readSummaryRequestData, requestTopicIdToNameMapping, describeShareGroupOffsetsResponseTopicList);
});
@@ -1917,18 +1916,20 @@ private CompletableFuture handleDeleteShare
.filter(errData -> errData.errorCode() != Errors.NONE.code())
.findAny();
+ String topicName = metadataImage.topicMetadata(topicData.topicId()).map(CoordinatorMetadataImage.TopicMetadata::name).orElse(null);
+
if (errItem.isPresent()) {
errorTopicResponses.add(
new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic()
.setTopicId(topicData.topicId())
- .setTopicName(metadataImage.topics().topicIdToNameView().get(topicData.topicId()))
+ .setTopicName(topicName)
.setErrorMessage(Errors.forCode(errItem.get().errorCode()).message())
.setErrorCode(errItem.get().errorCode())
);
} else {
successTopics.put(
topicData.topicId(),
- metadataImage.topics().topicIdToNameView().get(topicData.topicId())
+ topicName
);
}
});
@@ -2138,17 +2139,14 @@ public void onPartitionsDeleted(
// At this point the metadata will not have been updated
// with the deleted topics. However, we must guard against it.
- if (metadataImage == null || metadataImage.equals(MetadataImage.EMPTY)) {
+ if (metadataImage == null || metadataImage.equals(CoordinatorMetadataImage.EMPTY)) {
return;
}
- Set topicIds = new HashSet<>();
- for (TopicPartition tp : topicPartitions) {
- TopicImage image = metadataImage.topics().getTopic(tp.topic());
- if (image != null) {
- topicIds.add(image.id());
- }
- }
+ Set topicIds = topicPartitions.stream()
+ .filter(tp -> metadataImage.topicMetadata(tp.topic()).isPresent())
+ .map(tp -> metadataImage.topicMetadata(tp.topic()).get().id())
+ .collect(Collectors.toSet());
if (topicIds.isEmpty()) {
return;
@@ -2200,12 +2198,12 @@ public void onResignation(
}
/**
- * See {@link GroupCoordinator#onNewMetadataImage(MetadataImage, MetadataDelta)}.
+ * See {@link GroupCoordinator#onNewMetadataImage(CoordinatorMetadataImage, CoordinatorMetadataDelta)}.
*/
@Override
public void onNewMetadataImage(
- MetadataImage newImage,
- MetadataDelta delta
+ CoordinatorMetadataImage newImage,
+ CoordinatorMetadataDelta delta
) {
throwIfNotActive();
metadataImage = newImage;
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
index 938809c5e7b2b..c591f8d376768 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorShard.java
@@ -61,6 +61,8 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.common.runtime.CoordinatorExecutor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
@@ -117,8 +119,6 @@
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.common.ApiMessageAndVersion;
@@ -1088,8 +1088,8 @@ private void cancelGroupSizeCounter() {
* @param newImage The metadata image.
*/
@Override
- public void onLoaded(MetadataImage newImage) {
- MetadataDelta emptyDelta = new MetadataDelta(newImage);
+ public void onLoaded(CoordinatorMetadataImage newImage) {
+ CoordinatorMetadataDelta emptyDelta = newImage.emptyDelta();
groupMetadataManager.onNewMetadataImage(newImage, emptyDelta);
coordinatorMetrics.activateMetricsShard(metricsShard);
@@ -1113,7 +1113,7 @@ public void onUnloaded() {
* @param delta The delta image.
*/
@Override
- public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
+ public void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {
groupMetadataManager.onNewMetadataImage(newImage, delta);
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
index 078a1c39d5e38..4bb56c6b3f7a0 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupMetadataManager.java
@@ -80,6 +80,8 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.common.runtime.CoordinatorExecutor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.common.runtime.CoordinatorTimer;
@@ -160,11 +162,6 @@
import org.apache.kafka.coordinator.group.streams.topics.EndpointToPartitionsManager;
import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager;
import org.apache.kafka.coordinator.group.streams.topics.TopicConfigurationException;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsDelta;
-import org.apache.kafka.image.TopicsImage;
import org.apache.kafka.server.authorizer.Action;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import org.apache.kafka.server.authorizer.AuthorizationResult;
@@ -294,7 +291,7 @@ public static class Builder {
private CoordinatorExecutor executor = null;
private GroupCoordinatorConfig config = null;
private GroupConfigManager groupConfigManager = null;
- private MetadataImage metadataImage = null;
+ private CoordinatorMetadataImage metadataImage = null;
private ShareGroupPartitionAssignor shareGroupAssignor = null;
private GroupCoordinatorMetricsShard metrics;
private Optional> authorizerPlugin = null;
@@ -340,7 +337,7 @@ Builder withStreamsGroupAssignors(List streamsGroupAssignors) {
return this;
}
- Builder withMetadataImage(MetadataImage metadataImage) {
+ Builder withMetadataImage(CoordinatorMetadataImage metadataImage) {
this.metadataImage = metadataImage;
return this;
}
@@ -363,7 +360,7 @@ Builder withAuthorizerPlugin(Optional> authorizerPlugin) {
GroupMetadataManager build() {
if (logContext == null) logContext = new LogContext();
if (snapshotRegistry == null) snapshotRegistry = new SnapshotRegistry(logContext);
- if (metadataImage == null) metadataImage = MetadataImage.EMPTY;
+ if (metadataImage == null) metadataImage = CoordinatorMetadataImage.EMPTY;
if (time == null) time = Time.SYSTEM;
if (authorizerPlugin == null) authorizerPlugin = Optional.empty();
@@ -485,17 +482,17 @@ GroupMetadataManager build() {
/**
* The metadata image.
*/
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
/**
* The cache for topic hash value by topic name.
* A topic hash is calculated when there is a group subscribes to it.
- * A topic hash is removed when it's updated in MetadataImage or there is no group subscribes to it.
+ * A topic hash is removed when it's updated in CoordinatorMetadataImage or there is no group subscribes to it.
*/
private final Map topicHashCache;
/**
- * This tracks the version (or the offset) of the last metadata image
+ * This tracks the version of the last metadata image
* with newly created topics.
*/
private long lastMetadataImageWithNewTopics = -1L;
@@ -526,7 +523,7 @@ private GroupMetadataManager(
CoordinatorTimer timer,
CoordinatorExecutor executor,
GroupCoordinatorMetricsShard metrics,
- MetadataImage metadataImage,
+ CoordinatorMetadataImage metadataImage,
GroupCoordinatorConfig config,
GroupConfigManager groupConfigManager,
ShareGroupPartitionAssignor shareGroupAssignor,
@@ -560,7 +557,7 @@ private GroupMetadataManager(
/**
* @return The current metadata image used by the group metadata manager.
*/
- public MetadataImage image() {
+ public CoordinatorMetadataImage image() {
return metadataImage;
}
@@ -649,7 +646,7 @@ public List consumerGroupDescr
describedGroups.add(consumerGroup(groupId, committedOffset).asDescribedGroup(
committedOffset,
defaultConsumerGroupAssignor.name(),
- metadataImage.topics()
+ metadataImage
));
} catch (GroupIdNotFoundException exception) {
describedGroups.add(new ConsumerGroupDescribeResponseData.DescribedGroup()
@@ -681,7 +678,7 @@ public List shareGroupDescribe(
describedGroups.add(shareGroup(groupId, committedOffset).asDescribedGroup(
committedOffset,
shareGroupAssignor.name(),
- metadataImage.topics()
+ metadataImage
));
} catch (GroupIdNotFoundException exception) {
describedGroups.add(new ShareGroupDescribeResponseData.DescribedGroup()
@@ -1940,7 +1937,7 @@ private CoordinatorResult stream
if (reconfigureTopology || group.configuredTopology().isEmpty()) {
log.info("[GroupId {}][MemberId {}] Configuring the topology {}", groupId, memberId, updatedTopology);
- updatedConfiguredTopology = InternalTopicManager.configureTopics(logContext, metadataHash, updatedTopology, metadataImage.topics());
+ updatedConfiguredTopology = InternalTopicManager.configureTopics(logContext, metadataHash, updatedTopology, metadataImage);
group.setConfiguredTopology(updatedConfiguredTopology);
} else {
updatedConfiguredTopology = group.configuredTopology().get();
@@ -2471,7 +2468,7 @@ private CoordinatorResult classicGroupJoinToConsumerGro
group::currentPartitionEpoch,
targetAssignmentEpoch,
targetAssignment,
- toTopicPartitions(subscription.ownedPartitions(), metadataImage.topics()),
+ toTopicPartitions(subscription.ownedPartitions(), metadataImage),
records
);
@@ -2748,18 +2745,19 @@ Map subscribedTopicsChangeMap(String groupId, Set su
// Here will add any topics which are subscribed but not initialized and initializing
// topics whose timestamp indicates that they are older than delta elapsed.
subscriptionTopicNames.forEach(topicName -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topicName);
- if (topicImage != null) {
- Set alreadyInitializedPartSet = alreadyInitialized.containsKey(topicImage.id()) ? alreadyInitialized.get(topicImage.id()).partitions() : Set.of();
- if (alreadyInitializedPartSet.isEmpty() || alreadyInitializedPartSet.size() < topicImage.partitions().size()) {
- Set partitionSet = new HashSet<>(topicImage.partitions().keySet());
- partitionSet.removeAll(alreadyInitializedPartSet);
+ metadataImage.topicMetadata(topicName).ifPresent(topicMetadata -> {
+ Set alreadyInitializedPartSet = alreadyInitialized.containsKey(topicMetadata.id()) ? alreadyInitialized.get(topicMetadata.id()).partitions() : Set.of();
+ if (alreadyInitializedPartSet.isEmpty() || alreadyInitializedPartSet.size() < topicMetadata.partitionCount()) {
// alreadyInitialized contains all initialized topics and initializing topics which are less than delta old
// which means we are putting subscribed topics which are unseen or initializing for more than delta. But, we
// are also updating the timestamp here which means, old initializing will not be included repeatedly.
- topicPartitionChangeMap.computeIfAbsent(topicImage.id(), k -> new InitMapValue(topicImage.name(), partitionSet, curTimestamp));
+ topicPartitionChangeMap.computeIfAbsent(topicMetadata.id(), k -> {
+ Set partitionSet = IntStream.range(0, topicMetadata.partitionCount()).boxed().collect(Collectors.toCollection(HashSet::new));
+ partitionSet.removeAll(alreadyInitializedPartSet);
+ return new InitMapValue(topicMetadata.name(), partitionSet, curTimestamp);
+ });
}
- }
+ });
});
return topicPartitionChangeMap;
}
@@ -2839,7 +2837,8 @@ void addInitializingTopicsRecords(String groupId, List record
if (!currentDeleting.isEmpty()) {
finalInitializingMap.keySet().forEach(key -> {
if (currentDeleting.remove(key)) {
- log.warn("Initializing topic {} for share group {} found in deleting state as well, removing from deleting.", metadataImage.topics().getTopic(key).name(), groupId);
+ String topicName = metadataImage.topicMetadata(key).map(CoordinatorMetadataImage.TopicMetadata::name).orElse(null);
+ log.warn("Initializing topic {} for share group {} found in deleting state as well, removing from deleting.", topicName, groupId);
}
});
}
@@ -3213,7 +3212,7 @@ public static Map refreshRegularExpressions(
String groupId,
Logger log,
Time time,
- MetadataImage image,
+ CoordinatorMetadataImage image,
Optional> authorizerPlugin,
Set regexes
) {
@@ -3235,7 +3234,7 @@ public static Map refreshRegularExpressions(
}
}
- for (String topicName : image.topics().topicsByName().keySet()) {
+ for (String topicName : image.topicNames()) {
for (Pattern regex : compiledRegexes) {
if (regex.matcher(topicName).matches()) {
resolvedRegexes.get(regex.pattern()).add(topicName);
@@ -3249,7 +3248,7 @@ public static Map refreshRegularExpressions(
resolvedRegexes
);
- long version = image.provenance().lastContainedOffset();
+ long version = image.version();
Map result = new HashMap<>(resolvedRegexes.size());
for (Map.Entry> resolvedRegex : resolvedRegexes.entrySet()) {
result.put(
@@ -3259,7 +3258,7 @@ public static Map refreshRegularExpressions(
}
log.info("[GroupId {}] Scanned {} topics to refresh regular expressions {} in {}ms.",
- groupId, image.topics().topicsByName().size(), resolvedRegexes.keySet(),
+ groupId, image.topicNames().size(), resolvedRegexes.keySet(),
time.milliseconds() - startTimeMs);
return result;
@@ -3867,7 +3866,7 @@ private TasksTuple updateStreamsTargetAssignment(
int groupEpoch,
StreamsGroupMember updatedMember,
ConfiguredTopology configuredTopology,
- MetadataImage metadataImage,
+ CoordinatorMetadataImage metadataImage,
List records
) {
TaskAssignor assignor = streamsGroupAssignor(group.groupId());
@@ -4925,24 +4924,20 @@ public CoordinatorResult uninitializeShareGroupState(
}
private Map attachTopicName(Set topicIds) {
- TopicsImage topicsImage = metadataImage.topics();
Map finalMap = new HashMap<>();
for (Uuid topicId : topicIds) {
- TopicImage topicImage = topicsImage.getTopic(topicId);
- String topicName = (topicImage != null) ? topicImage.name() : "";
+ String topicName = metadataImage.topicMetadata(topicId).map(CoordinatorMetadataImage.TopicMetadata::name).orElse("");
finalMap.put(topicId, topicName);
}
return Collections.unmodifiableMap(finalMap);
}
private Map attachInitValue(Map> initMap) {
- TopicsImage topicsImage = metadataImage.topics();
Map finalMap = new HashMap<>();
long timestamp = time.milliseconds();
for (Map.Entry> entry : initMap.entrySet()) {
Uuid topicId = entry.getKey();
- TopicImage topicImage = topicsImage.getTopic(topicId);
- String topicName = (topicImage != null) ? topicImage.name() : "";
+ String topicName = metadataImage.topicMetadata(topicId).map(CoordinatorMetadataImage.TopicMetadata::name).orElse("");
finalMap.put(topicId, new InitMapValue(topicName, entry.getValue(), timestamp));
}
return Collections.unmodifiableMap(finalMap);
@@ -5774,41 +5769,37 @@ public void replay(
* @param newImage The new metadata image.
* @param delta The delta image.
*/
- public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
+ public void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {
metadataImage = newImage;
- // Initialize the last offset if it was not yet.
+ // Initialize the last version if it was not yet.
if (lastMetadataImageWithNewTopics == -1L) {
- lastMetadataImageWithNewTopics = metadataImage.provenance().lastContainedOffset();
+ lastMetadataImageWithNewTopics = metadataImage.version();
}
- TopicsDelta topicsDelta = delta.topicsDelta();
- if (topicsDelta == null) return;
-
- // Updated the last offset of the image with newly created topics. This is used to
+ // Updated the last version of the image with newly created topics. This is used to
// trigger a refresh of all the regular expressions when topics are created. Note
// that we don't trigger a refresh when topics are deleted. Those are removed from
// the subscription metadata (and the assignment) via the above mechanism. The
// resolved regular expressions are cleaned up on the next refresh.
- if (!topicsDelta.createdTopicIds().isEmpty()) {
- lastMetadataImageWithNewTopics = metadataImage.provenance().lastContainedOffset();
+ if (!delta.createdTopicIds().isEmpty()) {
+ lastMetadataImageWithNewTopics = metadataImage.version();
}
// Notify all the groups subscribed to the created, updated or
// deleted topics.
Set allGroupIds = new HashSet<>();
- topicsDelta.changedTopics().forEach((topicId, topicDelta) -> {
- String topicName = topicDelta.name();
- // Remove topic hash from the cache to recalculate it.
- topicHashCache.remove(topicName);
- allGroupIds.addAll(groupsSubscribedToTopic(topicName));
- });
- topicsDelta.deletedTopicIds().forEach(topicId -> {
- TopicImage topicImage = delta.image().topics().getTopic(topicId);
- String topicName = topicImage.name();
- topicHashCache.remove(topicName);
- allGroupIds.addAll(groupsSubscribedToTopic(topicName));
- });
+ delta.changedTopicIds().forEach(topicId ->
+ metadataImage.topicMetadata(topicId).ifPresent(topicMetadata -> {
+ // Remove topic hash from the cache to recalculate it.
+ topicHashCache.remove(topicMetadata.name());
+ allGroupIds.addAll(groupsSubscribedToTopic(topicMetadata.name()));
+ }));
+ delta.deletedTopicIds().forEach(topicId ->
+ delta.image().topicMetadata(topicId).ifPresent(topicMetadata -> {
+ topicHashCache.remove(topicMetadata.name());
+ allGroupIds.addAll(groupsSubscribedToTopic(topicMetadata.name()));
+ }));
allGroupIds.forEach(groupId -> {
Group group = groups.get(groupId);
if (group != null) {
@@ -7480,7 +7471,7 @@ private byte[] prepareAssignment(ConsumerGroupMember member) {
return ConsumerProtocol.serializeAssignment(
toConsumerProtocolAssignment(
member.assignedPartitions(),
- metadataImage.topics()
+ metadataImage
),
ConsumerProtocol.deserializeVersion(
ByteBuffer.wrap(member.classicMemberMetadata().get().supportedProtocols().iterator().next().metadata())
@@ -8041,20 +8032,23 @@ public Optional shareGroupBuildPartitionDeleteR
// a retry for the same is possible. Since this is part of an admin operation
// retrying delete should not pose issues related to
// performance. Also, the share coordinator is idempotent on delete partitions.
-
Set currentDeleting = shareGroupStatePartitionMetadata.get(shareGroupId).deletingTopics();
Map deleteRetryCandidates = new HashMap<>();
Set deletingToIgnore = new HashSet<>();
if (!currentDeleting.isEmpty()) {
- if (metadataImage == null || metadataImage.equals(MetadataImage.EMPTY)) {
+ if (metadataImage == null || metadataImage.equals(CoordinatorMetadataImage.EMPTY)) {
deletingToIgnore.addAll(currentDeleting);
} else {
for (Uuid deletingTopicId : currentDeleting) {
- TopicImage topicImage = metadataImage.topics().getTopic(deletingTopicId);
- if (topicImage == null) {
+ Optional topicMetadataOp = metadataImage.topicMetadata(deletingTopicId);
+ if (topicMetadataOp.isEmpty()) {
deletingToIgnore.add(deletingTopicId);
} else {
- deleteRetryCandidates.put(deletingTopicId, new InitMapValue(topicImage.name(), topicImage.partitions().keySet(), -1));
+ deleteRetryCandidates.put(deletingTopicId,
+ new InitMapValue(
+ topicMetadataOp.get().name(),
+ IntStream.range(0, topicMetadataOp.get().partitionCount()).boxed().collect(Collectors.toSet()),
+ -1));
}
}
}
@@ -8135,9 +8129,10 @@ public List sharePartitionsEli
Set deletingTopics = new HashSet<>(currentMap.deletingTopics());
requestData.topics().forEach(topic -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topic.topicName());
- if (topicImage != null) {
- Uuid topicId = topicImage.id();
+ Optional topicMetadataOpt = metadataImage.topicMetadata(topic.topicName());
+ if (topicMetadataOpt.isPresent()) {
+ var topicMetadata = topicMetadataOpt.get();
+ Uuid topicId = topicMetadata.id();
// A deleteState request to persister should only be sent with those topic partitions for which corresponding
// share partitions are initialized for the group.
if (initializedTopics.containsKey(topicId)) {
@@ -8157,7 +8152,7 @@ public List sharePartitionsEli
// If the topic for which delete share group offsets request is sent is already present in the deletingTopics set,
// we will include that topic in the delete share group state request.
List partitions = new ArrayList<>();
- topicImage.partitions().keySet().forEach(partition ->
+ IntStream.range(0, topicMetadata.partitionCount()).forEach(partition ->
partitions.add(new DeleteShareGroupStateRequestData.PartitionData().setPartition(partition)));
deleteShareGroupStateRequestTopicsData.add(
new DeleteShareGroupStateRequestData.DeleteStateData()
@@ -8205,13 +8200,13 @@ public Map.Entry> offsetByTopicPartitions = new HashMap<>();
alterShareGroupOffsetsRequest.topics().forEach(topic -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topic.topicName());
- if (topicImage != null) {
- Uuid topicId = topicImage.id();
- Set existingPartitions = new HashSet<>(topicImage.partitions().keySet());
+ Optional topicMetadataOpt = metadataImage.topicMetadata(topic.topicName());
+ if (topicMetadataOpt.isPresent()) {
+ var topicMetadata = topicMetadataOpt.get();
+ Uuid topicId = topicMetadata.id();
List partitions = new ArrayList<>();
topic.partitions().forEach(partition -> {
- if (existingPartitions.contains(partition.partitionIndex())) {
+ if (partition.partitionIndex() < topicMetadata.partitionCount()) {
partitions.add(
new AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
.setPartitionIndex(partition.partitionIndex())
@@ -8230,7 +8225,7 @@ public Map.Entry part < topicMetadata.partitionCount())
.collect(Collectors.toSet()),
currentTimeMs
));
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java
index 02b0ed28e6ec2..31e975779aef2 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/Utils.java
@@ -23,13 +23,9 @@
import org.apache.kafka.common.message.ConsumerProtocolAssignment;
import org.apache.kafka.common.message.ConsumerProtocolSubscription;
import org.apache.kafka.common.protocol.ApiMessage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
import org.apache.kafka.coordinator.group.generated.ShareGroupCurrentMemberAssignmentValue;
-import org.apache.kafka.image.ClusterImage;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsImage;
-import org.apache.kafka.metadata.BrokerRegistration;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import com.dynatrace.hash4j.hashing.HashStream64;
@@ -141,22 +137,20 @@ public static Optional toOptional(String str) {
* Converts a map of topic id and partition set to a ConsumerProtocolAssignment.
*
* @param assignment The map to convert.
- * @param topicsImage The TopicsImage.
+ * @param image The CoordinatorMetadataImage.
* @return The converted ConsumerProtocolAssignment.
*/
public static ConsumerProtocolAssignment toConsumerProtocolAssignment(
Map> assignment,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
ConsumerProtocolAssignment.TopicPartitionCollection collection =
new ConsumerProtocolAssignment.TopicPartitionCollection();
assignment.forEach((topicId, partitions) -> {
- TopicImage topicImage = topicsImage.getTopic(topicId);
- if (topicImage != null) {
+ image.topicMetadata(topicId).ifPresent(topicMetadata ->
collection.add(new ConsumerProtocolAssignment.TopicPartition()
- .setTopic(topicImage.name())
- .setPartitions(new ArrayList<>(partitions)));
- }
+ .setTopic(topicMetadata.name())
+ .setPartitions(new ArrayList<>(partitions))));
});
return new ConsumerProtocolAssignment()
.setAssignedPartitions(collection);
@@ -165,20 +159,19 @@ public static ConsumerProtocolAssignment toConsumerProtocolAssignment(
/**
* Converts a map of topic id and partition set to a ConsumerProtocolAssignment.
*
- * @param consumerProtocolAssignment The ConsumerProtocolAssignment.
- * @param topicsImage The TopicsImage.
+ * @param consumerProtocolAssignment The ConsumerProtocolAssignment.
+ * @param metadataImage The Metadata image.
* @return The converted map.
*/
public static Map> toTopicPartitionMap(
ConsumerProtocolAssignment consumerProtocolAssignment,
- TopicsImage topicsImage
+ CoordinatorMetadataImage metadataImage
) {
Map> topicPartitionMap = new HashMap<>();
consumerProtocolAssignment.assignedPartitions().forEach(topicPartition -> {
- TopicImage topicImage = topicsImage.getTopic(topicPartition.topic());
- if (topicImage != null) {
- topicPartitionMap.put(topicImage.id(), new HashSet<>(topicPartition.partitions()));
- }
+ metadataImage.topicMetadata(topicPartition.topic()).ifPresent(topicMetadata -> {
+ topicPartitionMap.put(topicMetadata.id(), new HashSet<>(topicPartition.partitions()));
+ });
});
return topicPartitionMap;
}
@@ -186,24 +179,23 @@ public static Map> toTopicPartitionMap(
/**
* Converts a ConsumerProtocolSubscription.TopicPartitionCollection to a list of ConsumerGroupHeartbeatRequestData.TopicPartitions.
*
- * @param topicPartitionCollection The TopicPartitionCollection to convert.
- * @param topicsImage The TopicsImage.
+ * @param topicPartitionCollection The TopicPartitionCollection to convert.
+ * @param metadataImage The Metadata image.
* @return a list of ConsumerGroupHeartbeatRequestData.TopicPartitions.
*/
public static List toTopicPartitions(
ConsumerProtocolSubscription.TopicPartitionCollection topicPartitionCollection,
- TopicsImage topicsImage
+ CoordinatorMetadataImage metadataImage
) {
List res = new ArrayList<>();
for (ConsumerProtocolSubscription.TopicPartition tp : topicPartitionCollection) {
- TopicImage topicImage = topicsImage.getTopic(tp.topic());
- if (topicImage != null) {
+ metadataImage.topicMetadata(tp.topic()).ifPresent(topicMetadata -> {
res.add(
new ConsumerGroupHeartbeatRequestData.TopicPartitions()
- .setTopicId(topicImage.id())
+ .setTopicId(topicMetadata.id())
.setPartitions(tp.partitions())
);
- }
+ });
}
return res;
}
@@ -398,38 +390,32 @@ public static long computeGroupHash(Map topicHashes) {
* 5. For each partition, write the partition ID and a sorted list of rack identifiers.
* - Rack identifiers are formatted as "" to prevent issues with simple separators.
*
- * @param topicName The topic image.
- * @param metadataImage The cluster image.
+ * @param topicName The topic name.
+ * @param metadataImage The topic metadata.
* @return The hash of the topic.
*/
- public static long computeTopicHash(String topicName, MetadataImage metadataImage) {
- TopicImage topicImage = metadataImage.topics().getTopic(topicName);
- if (topicImage == null) {
+ public static long computeTopicHash(String topicName, CoordinatorMetadataImage metadataImage) {
+ Optional topicImage = metadataImage.topicMetadata(topicName);
+ if (topicImage.isEmpty()) {
return 0;
}
+ CoordinatorMetadataImage.TopicMetadata topicMetadata = topicImage.get();
+
HashStream64 hasher = Hashing.xxh3_64().hashStream();
hasher = hasher
.putByte(TOPIC_HASH_MAGIC_BYTE)
- .putLong(topicImage.id().getMostSignificantBits())
- .putLong(topicImage.id().getLeastSignificantBits())
- .putString(topicImage.name())
- .putInt(topicImage.partitions().size());
-
- ClusterImage clusterImage = metadataImage.cluster();
- List racks = new ArrayList<>();
- for (int i = 0; i < topicImage.partitions().size(); i++) {
+ .putLong(topicMetadata.id().getMostSignificantBits())
+ .putLong(topicMetadata.id().getLeastSignificantBits())
+ .putString(topicMetadata.name())
+ .putInt(topicMetadata.partitionCount());
+
+ for (int i = 0; i < topicMetadata.partitionCount(); i++) {
hasher = hasher.putInt(i);
- racks.clear(); // Clear the list for reuse
- for (int replicaId : topicImage.partitions().get(i).replicas) {
- BrokerRegistration broker = clusterImage.broker(replicaId);
- if (broker != null) {
- broker.rack().ifPresent(racks::add);
- }
- }
+ List partitionRacks = topicMetadata.partitionRacks(i);
+ Collections.sort(partitionRacks);
- Collections.sort(racks);
- for (String rack : racks) {
+ for (String rack : partitionRacks) {
// Format: ""
// The rack string combination cannot use simple separator like ",", because there is no limitation for rack character.
// If using simple separator like "," it may hit edge case like ",," and ",,," / ",,," and ",,".
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
index 0e14d4e63db00..7a21a05e72f78 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/classic/ClassicGroup.java
@@ -33,6 +33,7 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
@@ -40,7 +41,6 @@
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
-import org.apache.kafka.image.MetadataImage;
import org.slf4j.Logger;
@@ -1343,14 +1343,14 @@ public Map groupAssignment() {
/**
* Convert the given ConsumerGroup to a corresponding ClassicGroup.
*
- * @param consumerGroup The converted ConsumerGroup.
- * @param leavingMembers The members that will not be converted in the ClassicGroup.
- * @param joiningMember The member that needs to be converted and added to the ClassicGroup.
- * When not null, must have an instanceId that matches an existing member.
- * @param logContext The logContext to create the ClassicGroup.
- * @param time The time to create the ClassicGroup.
- * @param metadataImage The MetadataImage.
- * @return The created ClassicGroup.
+ * @param consumerGroup The converted ConsumerGroup.
+ * @param leavingMembers The members that will not be converted in the ClassicGroup.
+ * @param joiningMember The member that needs to be converted and added to the ClassicGroup.
+ * When not null, must have an instanceId that matches an existing member.
+ * @param logContext The logContext to create the ClassicGroup.
+ * @param time The time to create the ClassicGroup.
+ * @param image The MetadataImage.
+ * @return The created ClassicGroup.
*/
public static ClassicGroup fromConsumerGroup(
ConsumerGroup consumerGroup,
@@ -1358,7 +1358,7 @@ public static ClassicGroup fromConsumerGroup(
ConsumerGroupMember joiningMember,
LogContext logContext,
Time time,
- MetadataImage metadataImage
+ CoordinatorMetadataImage image
) {
ClassicGroup classicGroup = new ClassicGroup(
logContext,
@@ -1427,7 +1427,7 @@ public static ClassicGroup fromConsumerGroup(
byte[] assignment = Utils.toArray(ConsumerProtocol.serializeAssignment(
toConsumerProtocolAssignment(
consumerGroup.targetAssignment().get(memberId).partitions(),
- metadataImage.topics()
+ image
),
ConsumerProtocol.deserializeVersion(
ByteBuffer.wrap(classicGroupMember.metadata(classicGroup.protocolName().orElse("")))
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
index c7e0815c8e625..1ae348e94dc1f 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/ModernGroup.java
@@ -19,11 +19,10 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.message.ListGroupsResponseData;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.Utils;
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
import org.apache.kafka.timeline.TimelineInteger;
@@ -36,6 +35,7 @@
import java.util.Map;
import java.util.Objects;
import java.util.Set;
+import java.util.stream.Collectors;
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HETEROGENEOUS;
import static org.apache.kafka.coordinator.group.api.assignor.SubscriptionType.HOMOGENEOUS;
@@ -367,18 +367,13 @@ public void setMetadataHash(long metadataHash) {
public static long computeMetadataHash(
Map subscribedTopicNames,
Map topicHashCache,
- MetadataImage metadataImage
+ CoordinatorMetadataImage metadataImage
) {
- Map topicHash = new HashMap<>(subscribedTopicNames.size());
- subscribedTopicNames.keySet().forEach(topicName -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topicName);
- if (topicImage != null) {
- topicHash.put(
- topicName,
- topicHashCache.computeIfAbsent(topicName, k -> Utils.computeTopicHash(topicName, metadataImage))
- );
- }
- });
+ Map topicHash = subscribedTopicNames.keySet().stream()
+ .filter(topicName -> metadataImage.topicMetadata(topicName).isPresent())
+ .collect(Collectors.toMap(
+ topicName -> topicName,
+ topicName -> topicHashCache.computeIfAbsent(topicName, k -> Utils.computeTopicHash(k, metadataImage))));
return Utils.computeGroupHash(topicHash);
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java
index 6be1bbfc99df4..a4c99ebc6c61a 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicDescriberImpl.java
@@ -17,16 +17,14 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor;
import org.apache.kafka.coordinator.group.api.assignor.SubscribedTopicDescriber;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.metadata.BrokerRegistration;
-import org.apache.kafka.metadata.PartitionRegistration;
-import java.util.Collections;
import java.util.HashSet;
+import java.util.List;
import java.util.Objects;
+import java.util.Optional;
import java.util.Set;
/**
@@ -37,9 +35,9 @@ public class SubscribedTopicDescriberImpl implements SubscribedTopicDescriber {
/**
* The metadata image that contains the latest metadata information.
*/
- private final MetadataImage metadataImage;
+ private final CoordinatorMetadataImage metadataImage;
- public SubscribedTopicDescriberImpl(MetadataImage metadataImage) {
+ public SubscribedTopicDescriberImpl(CoordinatorMetadataImage metadataImage) {
this.metadataImage = Objects.requireNonNull(metadataImage);
}
@@ -52,8 +50,7 @@ public SubscribedTopicDescriberImpl(MetadataImage metadataImage) {
*/
@Override
public int numPartitions(Uuid topicId) {
- TopicImage topicImage = this.metadataImage.topics().getTopic(topicId);
- return topicImage == null ? -1 : topicImage.partitions().size();
+ return this.metadataImage.topicMetadata(topicId).map(CoordinatorMetadataImage.TopicMetadata::partitionCount).orElse(-1);
}
/**
@@ -66,22 +63,18 @@ public int numPartitions(Uuid topicId) {
*/
@Override
public Set racksForPartition(Uuid topicId, int partition) {
- TopicImage topic = metadataImage.topics().getTopic(topicId);
- if (topic != null) {
- PartitionRegistration partitionRegistration = topic.partitions().get(partition);
- if (partitionRegistration != null) {
- Set racks = new HashSet<>();
- for (int replica : partitionRegistration.replicas) {
- // Only add the rack if it is available for the broker/replica.
- BrokerRegistration brokerRegistration = metadataImage.cluster().broker(replica);
- if (brokerRegistration != null) {
- brokerRegistration.rack().ifPresent(racks::add);
- }
- }
- return Collections.unmodifiableSet(racks);
- }
+ Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty()) {
+ return Set.of();
+ }
+
+ CoordinatorMetadataImage.TopicMetadata topicMetadata = topicMetadataOp.get();
+ List racks = topicMetadata.partitionRacks(partition);
+ if (racks == null) {
+ return Set.of();
+ } else {
+ return new HashSet<>(racks);
}
- return Set.of();
}
@Override
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java
index a46f89de396f4..8651216d84053 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilder.java
@@ -17,6 +17,7 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
@@ -27,7 +28,6 @@
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
-import org.apache.kafka.image.MetadataImage;
import java.util.ArrayList;
import java.util.Collections;
@@ -269,7 +269,7 @@ protected MemberSubscriptionAndAssignmentImpl newMemberSubscriptionAndAssignment
/**
* The metadata image.
*/
- private MetadataImage metadataImage = MetadataImage.EMPTY;
+ private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* The members which have been updated or deleted. Deleted members
@@ -376,7 +376,7 @@ public U withInvertedTargetAssignment(
* @return This object.
*/
public U withMetadataImage(
- MetadataImage metadataImage
+ CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return self();
@@ -427,7 +427,7 @@ public U removeMember(
*/
public TargetAssignmentResult build() throws PartitionAssignorException {
Map memberSpecs = new HashMap<>();
- TopicIds.TopicResolver topicResolver = new TopicIds.CachedTopicResolver(metadataImage.topics());
+ TopicIds.TopicResolver topicResolver = new TopicIds.CachedTopicResolver(metadataImage);
// Prepare the member spec for all members.
members.forEach((memberId, member) ->
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TopicIds.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TopicIds.java
index b1fa86a8ed560..9d71c9ef83140 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TopicIds.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/TopicIds.java
@@ -17,8 +17,7 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsImage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import java.util.Collection;
import java.util.HashMap;
@@ -40,7 +39,7 @@ public interface TopicResolver {
/**
* @return The TopicsImage used by the resolver.
*/
- TopicsImage image();
+ CoordinatorMetadataImage image();
/**
* Converts a topic id to a topic name.
@@ -70,31 +69,27 @@ public interface TopicResolver {
* A TopicResolver without any caching.
*/
public static class DefaultTopicResolver implements TopicResolver {
- private final TopicsImage image;
+ private final CoordinatorMetadataImage image;
public DefaultTopicResolver(
- TopicsImage image
+ CoordinatorMetadataImage image
) {
this.image = Objects.requireNonNull(image);
}
@Override
- public final TopicsImage image() {
+ public final CoordinatorMetadataImage image() {
return image;
}
@Override
public String name(Uuid id) {
- TopicImage topic = image.getTopic(id);
- if (topic == null) return null;
- return topic.name();
+ return image.topicMetadata(id).map(CoordinatorMetadataImage.TopicMetadata::name).orElse(null);
}
@Override
public Uuid id(String name) {
- TopicImage topic = image.getTopic(name);
- if (topic == null) return null;
- return topic.id();
+ return image.topicMetadata(name).map(CoordinatorMetadataImage.TopicMetadata::id).orElse(null);
}
@Override
@@ -113,38 +108,30 @@ public String toString() {
* TargetAssignmentBuilder.build() call.
*/
public static class CachedTopicResolver implements TopicResolver {
- private final TopicsImage image;
+ private final CoordinatorMetadataImage image;
private final Map topicIds = new HashMap<>();
private final Map topicNames = new HashMap<>();
public CachedTopicResolver(
- TopicsImage image
+ CoordinatorMetadataImage image
) {
this.image = Objects.requireNonNull(image);
}
@Override
- public final TopicsImage image() {
+ public final CoordinatorMetadataImage image() {
return image;
}
@Override
public String name(Uuid id) {
- return topicNames.computeIfAbsent(id, __ -> {
- TopicImage topic = image.getTopic(id);
- if (topic == null) return null;
- return topic.name();
- });
+ return topicNames.computeIfAbsent(id, __ -> image.topicMetadata(id).map(CoordinatorMetadataImage.TopicMetadata::name).orElse(null));
}
@Override
public Uuid id(String name) {
- return topicIds.computeIfAbsent(name, __ -> {
- TopicImage topic = image.getTopic(name);
- if (topic == null) return null;
- return topic.id();
- });
+ return topicIds.computeIfAbsent(name, __ -> image.topicMetadata(name).map(CoordinatorMetadataImage.TopicMetadata::id).orElse(null));
}
@Override
@@ -164,7 +151,7 @@ public String toString() {
public TopicIds(
Set topicNames,
- TopicsImage image
+ CoordinatorMetadataImage image
) {
this.topicNames = Objects.requireNonNull(topicNames);
this.resolver = new DefaultTopicResolver(image);
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
index 880cd49769c5e..c864836dece3d 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroup.java
@@ -29,6 +29,7 @@
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.types.SchemaException;
import org.apache.kafka.common.requests.JoinGroupRequest;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
@@ -43,8 +44,6 @@
import org.apache.kafka.coordinator.group.modern.ModernGroup;
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
import org.apache.kafka.coordinator.group.modern.SubscriptionCount;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicsImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
import org.apache.kafka.timeline.TimelineInteger;
@@ -1108,7 +1107,7 @@ void addPartitionEpochs(
public ConsumerGroupDescribeResponseData.DescribedGroup asDescribedGroup(
long committedOffset,
String defaultAssignor,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
ConsumerGroupDescribeResponseData.DescribedGroup describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
.setGroupId(groupId)
@@ -1120,7 +1119,7 @@ public ConsumerGroupDescribeResponseData.DescribedGroup asDescribedGroup(
entry -> describedGroup.members().add(
entry.getValue().asConsumerGroupDescribeMember(
targetAssignment.get(entry.getValue().memberId(), committedOffset),
- topicsImage
+ image
)
)
);
@@ -1145,7 +1144,7 @@ public static ConsumerGroup fromClassicGroup(
GroupCoordinatorMetricsShard metrics,
ClassicGroup classicGroup,
Map topicHashCache,
- MetadataImage metadataImage
+ CoordinatorMetadataImage metadataImage
) {
String groupId = classicGroup.groupId();
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics);
@@ -1165,7 +1164,7 @@ public static ConsumerGroup fromClassicGroup(
if (assignment.userData() != null && assignment.userData().hasRemaining()) {
throw new UnsupportedVersionException("userData from a custom assignor would be lost");
}
- assignedPartitions = toTopicPartitionMap(assignment, metadataImage.topics());
+ assignedPartitions = toTopicPartitionMap(assignment, metadataImage);
}
// Every member is guaranteed to have metadata set when it joins,
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java
index 4cc4895ff5cf1..9b5f0c1f6c80e 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMember.java
@@ -19,14 +19,13 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.Utils;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsImage;
import java.util.ArrayList;
import java.util.HashSet;
@@ -385,22 +384,22 @@ public Optional> supporte
/**
* @param targetAssignment The target assignment of this member in the corresponding group.
- *
+ * @param image
* @return The ConsumerGroupMember mapped as ConsumerGroupDescribeResponseData.Member.
*/
public ConsumerGroupDescribeResponseData.Member asConsumerGroupDescribeMember(
Assignment targetAssignment,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
return new ConsumerGroupDescribeResponseData.Member()
.setMemberEpoch(memberEpoch)
.setMemberId(memberId)
.setAssignment(new ConsumerGroupDescribeResponseData.Assignment()
- .setTopicPartitions(topicPartitionsFromMap(assignedPartitions, topicsImage)))
+ .setTopicPartitions(topicPartitionsFromMap(assignedPartitions, image)))
.setTargetAssignment(new ConsumerGroupDescribeResponseData.Assignment()
.setTopicPartitions(topicPartitionsFromMap(
targetAssignment != null ? targetAssignment.partitions() : Map.of(),
- topicsImage
+ image
)))
.setClientHost(clientHost)
.setClientId(clientId)
@@ -413,17 +412,14 @@ public ConsumerGroupDescribeResponseData.Member asConsumerGroupDescribeMember(
private static List topicPartitionsFromMap(
Map> partitions,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
List topicPartitions = new ArrayList<>();
partitions.forEach((topicId, partitionSet) -> {
- TopicImage topicImage = topicsImage.getTopic(topicId);
- if (topicImage != null) {
- topicPartitions.add(new ConsumerGroupDescribeResponseData.TopicPartitions()
- .setTopicId(topicId)
- .setTopicName(topicImage.name())
- .setPartitions(new ArrayList<>(partitionSet)));
- }
+ image.topicMetadata(topicId).ifPresent(topicMetadata -> topicPartitions.add(new ConsumerGroupDescribeResponseData.TopicPartitions()
+ .setTopicId(topicId)
+ .setTopicName(topicMetadata.name())
+ .setPartitions(new ArrayList<>(partitionSet))));
});
return topicPartitions;
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java
index e0fc1190a2eab..8a02e941008da 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroup.java
@@ -22,11 +22,11 @@
import org.apache.kafka.common.errors.UnknownMemberIdException;
import org.apache.kafka.common.message.ShareGroupDescribeResponseData;
import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.modern.ModernGroup;
-import org.apache.kafka.image.TopicsImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineObject;
@@ -309,7 +309,7 @@ protected void maybeUpdateGroupState() {
public ShareGroupDescribeResponseData.DescribedGroup asDescribedGroup(
long committedOffset,
String defaultAssignor,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
ShareGroupDescribeResponseData.DescribedGroup describedGroup = new ShareGroupDescribeResponseData.DescribedGroup()
.setGroupId(groupId)
@@ -320,7 +320,7 @@ public ShareGroupDescribeResponseData.DescribedGroup asDescribedGroup(
members.entrySet(committedOffset).forEach(
entry -> describedGroup.members().add(
entry.getValue().asShareGroupDescribeMember(
- topicsImage
+ image
)
)
);
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java
index e8d5b118b328d..57af4c98fd49c 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMember.java
@@ -18,13 +18,12 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ShareGroupDescribeResponseData;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.Utils;
import org.apache.kafka.coordinator.group.generated.ShareGroupCurrentMemberAssignmentValue;
import org.apache.kafka.coordinator.group.generated.ShareGroupMemberMetadataValue;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.ModernGroupMember;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsImage;
import java.util.ArrayList;
import java.util.HashSet;
@@ -198,18 +197,17 @@ private ShareGroupMember(
/**
* Converts this ShareGroupMember to a ShareGroupDescribeResponseData.Member.
*
- * @param topicsImage: Topics image object to search for a specific topic id
- *
+ * @param image : Topics image object to search for a specific topic id
* @return The ShareGroupMember mapped as ShareGroupDescribeResponseData.Member.
*/
public ShareGroupDescribeResponseData.Member asShareGroupDescribeMember(
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
return new ShareGroupDescribeResponseData.Member()
.setMemberEpoch(memberEpoch)
.setMemberId(memberId)
.setAssignment(new ShareGroupDescribeResponseData.Assignment()
- .setTopicPartitions(topicPartitionsFromMap(assignedPartitions, topicsImage)))
+ .setTopicPartitions(topicPartitionsFromMap(assignedPartitions, image)))
.setClientHost(clientHost)
.setClientId(clientId)
.setRackId(rackId)
@@ -218,17 +216,14 @@ public ShareGroupDescribeResponseData.Member asShareGroupDescribeMember(
private static List topicPartitionsFromMap(
Map> partitions,
- TopicsImage topicsImage
+ CoordinatorMetadataImage image
) {
List topicPartitions = new ArrayList<>();
partitions.forEach((topicId, partitionSet) -> {
- TopicImage topicImage = topicsImage.getTopic(topicId);
- if (topicImage != null) {
- topicPartitions.add(new ShareGroupDescribeResponseData.TopicPartitions()
- .setTopicId(topicId)
- .setTopicName(topicImage.name())
- .setPartitions(new ArrayList<>(partitionSet)));
- }
+ image.topicMetadata(topicId).ifPresent(topicMetadata -> topicPartitions.add(new ShareGroupDescribeResponseData.TopicPartitions()
+ .setTopicId(topicId)
+ .setTopicName(topicMetadata.name())
+ .setPartitions(new ArrayList<>(partitionSet))));
});
return topicPartitions;
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java
index afc252a7fee29..b75f3926d0898 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/StreamsGroup.java
@@ -25,6 +25,7 @@
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.requests.JoinGroupRequest;
import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
@@ -33,8 +34,6 @@
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetricsShard;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.apache.kafka.timeline.TimelineHashMap;
import org.apache.kafka.timeline.TimelineInteger;
@@ -618,7 +617,7 @@ public void setMetadataHash(long metadataHash) {
* @return The metadata hash.
*/
public long computeMetadataHash(
- MetadataImage metadataImage,
+ CoordinatorMetadataImage metadataImage,
Map topicHashCache,
StreamsTopology topology
) {
@@ -626,13 +625,11 @@ public long computeMetadataHash(
Map topicHash = new HashMap<>(requiredTopicNames.size());
requiredTopicNames.forEach(topicName -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topicName);
- if (topicImage != null) {
+ metadataImage.topicMetadata(topicName).ifPresent(__ ->
topicHash.put(
topicName,
topicHashCache.computeIfAbsent(topicName, k -> Utils.computeTopicHash(topicName, metadataImage))
- );
- }
+ ));
});
return Utils.computeGroupHash(topicHash);
}
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilder.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilder.java
index 7f8b504bab9f3..809489907baa4 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilder.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilder.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.coordinator.group.streams;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.streams.assignor.AssignmentMemberSpec;
import org.apache.kafka.coordinator.group.streams.assignor.GroupAssignment;
@@ -24,7 +25,6 @@
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor;
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignorException;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
-import org.apache.kafka.image.MetadataImage;
import java.util.ArrayList;
import java.util.Collections;
@@ -78,7 +78,7 @@ public class TargetAssignmentBuilder {
/**
* The metadata image.
*/
- private MetadataImage metadataImage = MetadataImage.EMPTY;
+ private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* The existing target assignment.
@@ -164,7 +164,7 @@ public TargetAssignmentBuilder withStaticMembers(
* @return This object.
*/
public TargetAssignmentBuilder withMetadataImage(
- MetadataImage metadataImage
+ CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return this;
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TopologyMetadata.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TopologyMetadata.java
index b91bfc98706a9..4e39db3126856 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TopologyMetadata.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/TopologyMetadata.java
@@ -16,9 +16,9 @@
*/
package org.apache.kafka.coordinator.group.streams;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.streams.assignor.TopologyDescriber;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
-import org.apache.kafka.image.MetadataImage;
import java.util.Collections;
import java.util.List;
@@ -33,7 +33,7 @@
* @param metadataImage The metadata image
* @param subtopologyMap The configured subtopologies
*/
-public record TopologyMetadata(MetadataImage metadataImage, SortedMap subtopologyMap) implements TopologyDescriber {
+public record TopologyMetadata(CoordinatorMetadataImage metadataImage, SortedMap subtopologyMap) implements TopologyDescriber {
public TopologyMetadata {
metadataImage = Objects.requireNonNull(metadataImage);
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java
index 09876efd8046a..316ac4040d7c4 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManager.java
@@ -18,16 +18,16 @@
package org.apache.kafka.coordinator.group.streams.topics;
import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.streams.StreamsGroup;
import org.apache.kafka.coordinator.group.streams.StreamsGroupMember;
-import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicImage;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Optional;
import java.util.Set;
public class EndpointToPartitionsManager {
@@ -38,7 +38,7 @@ private EndpointToPartitionsManager() {
public static StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToPartitions(final StreamsGroupMember streamsGroupMember,
final StreamsGroupHeartbeatResponseData.Endpoint responseEndpoint,
final StreamsGroup streamsGroup,
- final MetadataImage metadataImage) {
+ final CoordinatorMetadataImage metadataImage) {
StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToPartitions = new StreamsGroupHeartbeatResponseData.EndpointToPartitions();
Map> activeTasks = streamsGroupMember.assignedTasks().activeTasks();
Map> standbyTasks = streamsGroupMember.assignedTasks().standbyTasks();
@@ -53,7 +53,7 @@ public static StreamsGroupHeartbeatResponseData.EndpointToPartitions endpointToP
private static List topicPartitions(final Map> tasks,
final Map configuredSubtopologies,
- final MetadataImage metadataImage) {
+ final CoordinatorMetadataImage metadataImage) {
List topicPartitionsForTasks = new ArrayList<>();
for (Map.Entry> taskEntry : tasks.entrySet()) {
String subtopologyId = taskEntry.getKey();
@@ -70,13 +70,13 @@ private static List topicParti
private static List topicPartitionListForTask(final Set taskSet,
final Set topicNames,
- final MetadataImage metadataImage) {
+ final CoordinatorMetadataImage metadataImage) {
return topicNames.stream().map(topic -> {
- TopicImage topicImage = metadataImage.topics().getTopic(topic);
- if (topicImage == null) {
+ Optional topicMetadata = metadataImage.topicMetadata(topic);
+ if (topicMetadata.isEmpty()) {
throw new IllegalStateException("Topic " + topic + " not found in metadata image");
}
- int numPartitionsForTopic = topicImage.partitions().size();
+ int numPartitionsForTopic = topicMetadata.get().partitionCount();
StreamsGroupHeartbeatResponseData.TopicPartition tp = new StreamsGroupHeartbeatResponseData.TopicPartition();
tp.setTopic(topic);
List tpPartitions = new ArrayList<>(taskSet);
diff --git a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java
index c1845ca2e24ec..5c89c622a248d 100644
--- a/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java
+++ b/group-coordinator/src/main/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManager.java
@@ -20,10 +20,9 @@
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicConfig;
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicConfigCollection;
import org.apache.kafka.common.utils.LogContext;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue;
import org.apache.kafka.coordinator.group.streams.StreamsTopology;
-import org.apache.kafka.image.TopicImage;
-import org.apache.kafka.image.TopicsImage;
import org.slf4j.Logger;
@@ -47,19 +46,19 @@
public class InternalTopicManager {
/**
- * Configures the internal topics for the given topology. Given a topology and the topics image, this method determines the number of
+ * Configures the internal topics for the given topology. Given a topology and the metadata image, this method determines the number of
* partitions for all internal topics and returns a {@link ConfiguredTopology} object.
*
- * @param logContext The log context.
- * @param metadataHash The metadata hash of the group.
- * @param topology The topology.
- * @param topicsImage The topics image.
+ * @param logContext The log context.
+ * @param metadataHash The metadata hash of the group.
+ * @param topology The topology.
+ * @param metadataImage The metadata image.
* @return The configured topology.
*/
public static ConfiguredTopology configureTopics(LogContext logContext,
long metadataHash,
StreamsTopology topology,
- TopicsImage topicsImage) {
+ CoordinatorMetadataImage metadataImage) {
final Logger log = logContext.logger(InternalTopicManager.class);
final Collection subtopologies = topology.subtopologies().values();
@@ -73,23 +72,23 @@ public static ConfiguredTopology configureTopics(LogContext logContext,
try {
Optional topicConfigurationException = Optional.empty();
- throwOnMissingSourceTopics(topology, topicsImage);
+ throwOnMissingSourceTopics(topology, metadataImage);
Map decidedPartitionCountsForInternalTopics =
- decidePartitionCounts(logContext, topology, topicsImage, copartitionGroupsBySubtopology, log);
+ decidePartitionCounts(logContext, topology, metadataImage, copartitionGroupsBySubtopology, log);
final SortedMap configuredSubtopologies =
subtopologies.stream()
.collect(Collectors.toMap(
StreamsGroupTopologyValue.Subtopology::subtopologyId,
- x -> fromPersistedSubtopology(x, topicsImage, decidedPartitionCountsForInternalTopics),
+ x -> fromPersistedSubtopology(x, metadataImage, decidedPartitionCountsForInternalTopics),
(v1, v2) -> {
throw new RuntimeException(String.format("Duplicate key for values %s and %s", v1, v2));
},
TreeMap::new
));
- Map internalTopicsToCreate = missingInternalTopics(configuredSubtopologies, topology, topicsImage);
+ Map internalTopicsToCreate = missingInternalTopics(configuredSubtopologies, topology, metadataImage);
if (!internalTopicsToCreate.isEmpty()) {
topicConfigurationException = Optional.of(TopicConfigurationException.missingInternalTopics(
"Internal topics are missing: " + internalTopicsToCreate.keySet()
@@ -122,11 +121,11 @@ public static ConfiguredTopology configureTopics(LogContext logContext,
}
private static void throwOnMissingSourceTopics(final StreamsTopology topology,
- final TopicsImage topicsImage) {
+ final CoordinatorMetadataImage metadataImage) {
TreeSet sortedMissingTopics = new TreeSet<>();
for (StreamsGroupTopologyValue.Subtopology subtopology : topology.subtopologies().values()) {
for (String sourceTopic : subtopology.sourceTopics()) {
- if (topicsImage.getTopic(sourceTopic) == null) {
+ if (metadataImage.topicMetadata(sourceTopic).isEmpty()) {
sortedMissingTopics.add(sourceTopic);
}
}
@@ -139,12 +138,12 @@ private static void throwOnMissingSourceTopics(final StreamsTopology topology,
private static Map decidePartitionCounts(final LogContext logContext,
final StreamsTopology topology,
- final TopicsImage topicsImage,
+ final CoordinatorMetadataImage metadataImage,
final Map>> copartitionGroupsBySubtopology,
final Logger log) {
final Map decidedPartitionCountsForInternalTopics = new HashMap<>();
final Function topicPartitionCountProvider =
- topic -> getPartitionCount(topicsImage, topic, decidedPartitionCountsForInternalTopics);
+ topic -> getPartitionCount(metadataImage, topic, decidedPartitionCountsForInternalTopics);
final RepartitionTopics repartitionTopics = new RepartitionTopics(
logContext,
topology.subtopologies().values(),
@@ -196,7 +195,7 @@ private static void enforceCopartitioning(final StreamsTopology topology,
private static Map missingInternalTopics(Map subtopologyMap,
StreamsTopology topology,
- TopicsImage topicsImage) {
+ CoordinatorMetadataImage metadataImage) {
final Map topicsToCreate = new HashMap<>();
for (ConfiguredSubtopology subtopology : subtopologyMap.values()) {
@@ -206,33 +205,31 @@ private static Map missingInternalTopics(Map topicsToCreate.put(x.name(), toCreatableTopic(x)));
}
for (String topic : topology.requiredTopics()) {
- TopicImage topicImage = topicsImage.getTopic(topic);
- if (topicImage == null) {
- continue;
- }
- final CreatableTopic expectedTopic = topicsToCreate.remove(topic);
- if (expectedTopic != null) {
- if (topicImage.partitions().size() != expectedTopic.numPartitions()) {
- throw TopicConfigurationException.incorrectlyPartitionedTopics("Existing topic " + topic + " has different"
- + " number of partitions: expected " + expectedTopic.numPartitions() + ", found " + topicImage.partitions().size());
+ metadataImage.topicMetadata(topic).ifPresent(topicMetadata -> {
+ final CreatableTopic expectedTopic = topicsToCreate.remove(topic);
+ if (expectedTopic != null) {
+ if (topicMetadata.partitionCount() != expectedTopic.numPartitions()) {
+ throw TopicConfigurationException.incorrectlyPartitionedTopics("Existing topic " + topic + " has different"
+ + " number of partitions: expected " + expectedTopic.numPartitions() + ", found " + topicMetadata.partitionCount());
+ }
}
- }
+ });
}
return topicsToCreate;
}
- private static OptionalInt getPartitionCount(TopicsImage topicsImage,
+ private static OptionalInt getPartitionCount(CoordinatorMetadataImage metadataImage,
String topic,
Map decidedPartitionCountsForInternalTopics) {
- final TopicImage topicImage = topicsImage.getTopic(topic);
- if (topicImage == null) {
+ Optional topicMetadata = metadataImage.topicMetadata(topic);
+ if (topicMetadata.isEmpty()) {
if (decidedPartitionCountsForInternalTopics.containsKey(topic)) {
return OptionalInt.of(decidedPartitionCountsForInternalTopics.get(topic));
} else {
return OptionalInt.empty();
}
} else {
- return OptionalInt.of(topicImage.partitions().size());
+ return OptionalInt.of(topicMetadata.get().partitionCount());
}
}
@@ -264,11 +261,11 @@ private static CreatableTopic toCreatableTopic(final ConfiguredInternalTopic con
}
private static ConfiguredSubtopology fromPersistedSubtopology(final StreamsGroupTopologyValue.Subtopology subtopology,
- final TopicsImage topicsImage,
+ final CoordinatorMetadataImage metadataImage,
final Map decidedPartitionCountsForInternalTopics
) {
return new ConfiguredSubtopology(
- computeNumberOfTasks(subtopology, topicsImage, decidedPartitionCountsForInternalTopics),
+ computeNumberOfTasks(subtopology, metadataImage, decidedPartitionCountsForInternalTopics),
new HashSet<>(subtopology.sourceTopics()),
subtopology.repartitionSourceTopics().stream()
.map(x -> fromPersistedTopicInfo(x, decidedPartitionCountsForInternalTopics))
@@ -281,13 +278,13 @@ private static ConfiguredSubtopology fromPersistedSubtopology(final StreamsGroup
}
private static int computeNumberOfTasks(final StreamsGroupTopologyValue.Subtopology subtopology,
- final TopicsImage topicsImage,
+ final CoordinatorMetadataImage metadataImage,
final Map decidedPartitionCountsForInternalTopics) {
return Stream.concat(
subtopology.sourceTopics().stream(),
subtopology.repartitionSourceTopics().stream().map(StreamsGroupTopologyValue.TopicInfo::name)
).map(
- topic -> getPartitionCount(topicsImage, topic, decidedPartitionCountsForInternalTopics).orElseThrow(
+ topic -> getPartitionCount(metadataImage, topic, decidedPartitionCountsForInternalTopics).orElseThrow(
() -> new IllegalStateException("Number of partitions must be set for topic " + topic)
)
).max(Integer::compareTo).orElseThrow(
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
index ba6ca2e34b0e7..01832467b0fbc 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorServiceTest.java
@@ -87,13 +87,16 @@
import org.apache.kafka.common.utils.BufferSupplier;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.metrics.GroupCoordinatorMetrics;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
-import org.apache.kafka.image.TopicsImage;
import org.apache.kafka.server.authorizer.AuthorizableRequestContext;
import org.apache.kafka.server.record.BrokerCompressionType;
import org.apache.kafka.server.share.persister.DefaultStatePersister;
@@ -3152,7 +3155,7 @@ public void testOnPartitionsDeleted() {
.addTopic(Uuid.randomUuid(), "foo", 1)
.build();
- service.onNewMetadataImage(image, new MetadataDelta(image));
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(new MetadataDelta(image)));
when(runtime.scheduleWriteAllOperation(
ArgumentMatchers.eq("on-partition-deleted"),
@@ -3210,7 +3213,7 @@ public void testOnPartitionsDeletedCleanupShareGroupState() {
.addTopic(Uuid.randomUuid(), "foo", 1)
.build();
- service.onNewMetadataImage(image, new MetadataDelta(image));
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(new MetadataDelta(image)));
// No error in partition deleted callback
when(runtime.scheduleWriteAllOperation(
@@ -3257,10 +3260,10 @@ public void testOnPartitionsDeletedCleanupShareGroupStateEmptyMetadata() {
.build();
service.startup(() -> 3);
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "bar", 1)
- .build();
- service.onNewMetadataImage(image, new MetadataDelta(image));
+ .buildCoordinatorMetadataImage();
+ service.onNewMetadataImage(image, image.emptyDelta());
// No error in partition deleted callback
when(runtime.scheduleWriteAllOperation(
@@ -3307,8 +3310,8 @@ public void testOnPartitionsDeletedCleanupShareGroupStateTopicsNotInMetadata() {
.build();
service.startup(() -> 3);
- MetadataImage image = MetadataImage.EMPTY;
- service.onNewMetadataImage(image, new MetadataDelta(image));
+ CoordinatorMetadataImage image = CoordinatorMetadataImage.EMPTY;
+ service.onNewMetadataImage(image, image.emptyDelta());
// No error in partition deleted callback
when(runtime.scheduleWriteAllOperation(
@@ -3971,7 +3974,7 @@ public void testDescribeShareGroupAllOffsets() throws InterruptedException, Exec
.addTopic(TOPIC_ID, TOPIC_NAME, 3)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int partition = 1;
@@ -4040,7 +4043,7 @@ public void testDescribeShareGroupAllOffsetsThrowsError() {
.addTopic(TOPIC_ID, TOPIC_NAME, 3)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int partition = 1;
@@ -4076,7 +4079,7 @@ public void testDescribeShareGroupAllOffsetsNullResult() {
.addTopic(TOPIC_ID, TOPIC_NAME, 3)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int partition = 1;
@@ -5204,7 +5207,7 @@ public void testPersisterInitializeSuccess() {
.addTopic(topicId, "topic-name", 3)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
when(mockPersister.initializeState(ArgumentMatchers.any())).thenReturn(CompletableFuture.completedFuture(
new InitializeShareGroupStateResult.Builder()
@@ -5379,7 +5382,7 @@ public void testPersisterInitializeGroupInitializeFailure() {
.addTopic(topicId, "topic-name", 3)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
when(mockPersister.initializeState(ArgumentMatchers.any())).thenReturn(CompletableFuture.completedFuture(
new InitializeShareGroupStateResult.Builder()
@@ -5597,7 +5600,7 @@ public void testPersisterInitializeForAlterShareGroupOffsetsResponseSuccess() {
.addTopic(topicId, "topic-name", 1)
.build();
- service.onNewMetadataImage(image, null);
+ service.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
when(mockPersister.initializeState(ArgumentMatchers.any())).thenReturn(CompletableFuture.completedFuture(
new InitializeShareGroupStateResult.Builder()
@@ -5646,7 +5649,7 @@ private static class GroupCoordinatorServiceBuilder {
private CoordinatorRuntime runtime;
private GroupCoordinatorMetrics metrics = new GroupCoordinatorMetrics();
private Persister persister = new NoOpStatePersister();
- private MetadataImage metadataImage = null;
+ private CoordinatorMetadataImage metadataImage = null;
GroupCoordinatorService build() {
return build(false);
@@ -5654,7 +5657,7 @@ GroupCoordinatorService build() {
GroupCoordinatorService build(boolean serviceStartup) {
if (metadataImage == null) {
- metadataImage = mock(MetadataImage.class);
+ metadataImage = mock(CoordinatorMetadataImage.class);
}
GroupCoordinatorService service = new GroupCoordinatorService(
@@ -5671,9 +5674,12 @@ GroupCoordinatorService build(boolean serviceStartup) {
service.startup(() -> 1);
service.onNewMetadataImage(metadataImage, null);
}
- when(metadataImage.topics()).thenReturn(mock(TopicsImage.class));
- when(metadataImage.topics().topicIdToNameView()).thenReturn(Map.of(TOPIC_ID, TOPIC_NAME));
- when(metadataImage.topics().topicNameToIdView()).thenReturn(Map.of(TOPIC_NAME, TOPIC_ID));
+ when(metadataImage.topicNames()).thenReturn(Set.of(TOPIC_NAME));
+ var topicMetadata = mock(CoordinatorMetadataImage.TopicMetadata.class);
+ when(topicMetadata.name()).thenReturn(TOPIC_NAME);
+ when(topicMetadata.id()).thenReturn(TOPIC_ID);
+ when(metadataImage.topicMetadata(TOPIC_ID)).thenReturn(Optional.of(topicMetadata));
+ when(metadataImage.topicMetadata(TOPIC_NAME)).thenReturn(Optional.of(topicMetadata));
return service;
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
index d58c6b6b1ac75..467b2cce28889 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupCoordinatorShardTest.java
@@ -42,6 +42,7 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
@@ -87,7 +88,6 @@
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroup;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup;
import org.apache.kafka.coordinator.group.streams.StreamsGroupHeartbeatResult;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.share.persister.DeleteShareGroupStateParameters;
import org.apache.kafka.server.share.persister.GroupTopicPartitionData;
@@ -1248,7 +1248,7 @@ public void testReplayKeyCannotBeNull() {
@Test
public void testOnLoaded() {
- MetadataImage image = MetadataImage.EMPTY;
+ CoordinatorMetadataImage image = CoordinatorMetadataImage.EMPTY;
GroupMetadataManager groupMetadataManager = mock(GroupMetadataManager.class);
OffsetMetadataManager offsetMetadataManager = mock(OffsetMetadataManager.class);
CoordinatorMetrics coordinatorMetrics = mock(CoordinatorMetrics.class);
@@ -1344,7 +1344,7 @@ public void testScheduleCleanupGroupMetadata() {
mock(CoordinatorMetrics.class),
mock(CoordinatorMetricsShard.class)
);
- MetadataImage image = MetadataImage.EMPTY;
+ CoordinatorMetadataImage image = CoordinatorMetadataImage.EMPTY;
// Confirm the cleanup is scheduled when the coordinator is initially loaded.
coordinator.onLoaded(image);
@@ -1479,7 +1479,7 @@ public void testScheduleGroupSizeCounter() {
coordinatorMetrics,
metricsShard
);
- coordinator.onLoaded(MetadataImage.EMPTY);
+ coordinator.onLoaded(CoordinatorMetadataImage.EMPTY);
// The counter is scheduled.
assertEquals(
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
index 5721e3ead8711..d4adb9e02a089 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTest.java
@@ -79,8 +79,13 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorExecutor;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorTimer.ExpiredTimeout;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorTimer.ScheduledTimeout;
@@ -292,7 +297,7 @@ public void testMemberIdGeneration() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -479,11 +484,11 @@ public void testMemberJoinsEmptyConsumerGroup() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -568,10 +573,10 @@ public void testTopicHashIsRemoveFromCacheIfNoGroupSubscribesIt() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
@@ -674,11 +679,11 @@ public void testUpdatingSubscriptionTriggersNewTargetAssignment() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -781,7 +786,7 @@ public void testNewRacksDataInMetadataImageTriggersEpochBump() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -798,7 +803,7 @@ public void testNewRacksDataInMetadataImageTriggersEpochBump() {
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage))
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)))
)))
.build();
@@ -809,14 +814,14 @@ fooTopicName, computeTopicHash(fooTopicName, metadataImage))
));
// Update metadata image with racks.
- MetadataImage newMetadataImage = new MetadataImageBuilder(metadataImage)
+ CoordinatorMetadataImage newMetadataImage = new MetadataImageBuilder(metadataImage)
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
context.groupMetadataManager.onNewMetadataImage(
newMetadataImage,
- new MetadataDelta(newMetadataImage)
+ newMetadataImage.emptyDelta()
);
// If a topic is updated, related topic hash is cleanup.
assertEquals(Map.of(), context.groupMetadataManager.topicHashCache());
@@ -874,7 +879,7 @@ public void testRemoveTopicCleanupTopicHash() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -892,7 +897,7 @@ public void testRemoveTopicCleanupTopicHash() {
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
fooTopicName,
- computeTopicHash(fooTopicName, metadataImage))
+ computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)))
)))
.build();
@@ -906,8 +911,8 @@ public void testRemoveTopicCleanupTopicHash() {
MetadataImage newMetadataImage = delta.apply(MetadataProvenance.EMPTY);
context.groupMetadataManager.onNewMetadataImage(
- newMetadataImage,
- new MetadataDelta(newMetadataImage)
+ new KRaftCoordinatorMetadataImage(newMetadataImage),
+ new KRaftCoordinatorMetadataDelta(new MetadataDelta(newMetadataImage))
);
// If a topic is removed, related topic hash is cleanup.
assertEquals(Map.of(), context.groupMetadataManager.topicHashCache());
@@ -966,7 +971,7 @@ public void testSubscriptionUpgradeToMetadataHash() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -1029,7 +1034,7 @@ public void testSubscriptionUpgradeToMetadataHash() {
List expectedRecords = List.of(
GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
))),
GroupCoordinatorRecordHelpers.newConsumerGroupSubscriptionMetadataTombstoneRecord(groupId),
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentEpochRecord(groupId, 11),
@@ -1058,14 +1063,14 @@ public void testNewJoiningMemberTriggersNewTargetAssignment() {
.addRacks()
.build();
long groupMetadataHash = computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage),
- barTopicName, computeTopicHash(barTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)),
+ barTopicName, computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
));
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -1195,14 +1200,14 @@ public void testLeavingMemberBumpsGroupEpoch() {
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
.build();
- long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
- long barTopicHash = computeTopicHash(barTopicName, metadataImage);
- long zarTopicHash = computeTopicHash(zarTopicName, metadataImage);
+ long fooTopicHash = computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage));
+ long barTopicHash = computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage));
+ long zarTopicHash = computeTopicHash(zarTopicName, new KRaftCoordinatorMetadataImage(metadataImage));
// Consumer group with two members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -1295,14 +1300,14 @@ public void testGroupEpochBumpWhenNewStaticMemberJoins() {
.addRacks()
.build();
long groupMetadataHash = computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage),
- barTopicName, computeTopicHash(barTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)),
+ barTopicName, computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
));
// Consumer group with two static members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -1466,7 +1471,7 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() {
// Consumer group with two static members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(member1)
.withMember(member2)
@@ -1478,8 +1483,8 @@ public void testStaticMemberGetsBackAssignmentUponRejoin() {
mkTopicAssignment(barTopicId, 2)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage),
- barTopicName, computeTopicHash(barTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)),
+ barTopicName, computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
))))
.build();
@@ -1637,13 +1642,13 @@ public void testStaticMemberRejoinsWithNewSubscribedTopics() {
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
.build();
- long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
- long barTopicHash = computeTopicHash(barTopicName, metadataImage);
+ long fooTopicHash = computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage));
+ long barTopicHash = computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage));
// Consumer group with two static members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(member1)
.withMember(member2)
@@ -1832,7 +1837,7 @@ public void testNoGroupEpochBumpWhenStaticMemberTemporarilyLeaves() {
// Consumer group with two static members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(member1)
.withMember(member2)
@@ -1844,8 +1849,8 @@ public void testNoGroupEpochBumpWhenStaticMemberTemporarilyLeaves() {
mkTopicAssignment(barTopicId, 2)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage),
- barTopicName, computeTopicHash(barTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)),
+ barTopicName, computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
))))
.build();
@@ -1893,12 +1898,12 @@ public void testLeavingStaticMemberBumpsGroupEpoch() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
long zarTopicHash = computeTopicHash(zarTopicName, metadataImage);
@@ -1993,10 +1998,10 @@ public void testShouldThrownUnreleasedInstanceIdExceptionWhenNewMemberJoinsWithI
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with one static member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -2049,9 +2054,9 @@ public void testShouldThrownUnknownMemberIdExceptionWhenUnknownStaticMemberJoins
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with one static member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -2103,9 +2108,9 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with one static member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -2242,9 +2247,9 @@ public void testShouldThrowUnknownMemberIdExceptionWhenUnknownStaticMemberLeaves
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with one static member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -2294,9 +2299,9 @@ public void testShouldThrowFencedInstanceIdExceptionWhenStaticMemberWithDifferen
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with one static member.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -2350,7 +2355,7 @@ public void testConsumerGroupHeartbeatFullResponse() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Prepare new assignment for the group.
@@ -2463,11 +2468,11 @@ public void testReconciliationProcess() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Create a context with one consumer group containing two members.
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
@@ -2900,10 +2905,10 @@ public void testNewMemberIsRejectedWithMaximumMembersIsReached() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
// Create a context with one consumer group containing two members.
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
@@ -2970,9 +2975,9 @@ public void testConsumerGroupStates() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -3038,7 +3043,7 @@ public void testPartitionAssignorExceptionOnRegularHeartbeat() {
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Member 1 joins the consumer group. The request fails because the
@@ -3064,10 +3069,10 @@ public void testSubscriptionMetadataRefreshedAfterGroupIsLoaded() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Create a context with one consumer group containing one member.
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
@@ -3096,7 +3101,7 @@ fooTopicName, computeTopicHash(fooTopicName, new MetadataImageBuilder()
// 6 partitions the metadata image.
.addTopic(fooTopicId, fooTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
))))
.build();
@@ -3174,10 +3179,10 @@ public void testSubscriptionMetadataRefreshedAgainAfterWriteFailure() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Create a context with one consumer group containing one member.
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
@@ -3206,7 +3211,7 @@ fooTopicName, computeTopicHash(fooTopicName, new MetadataImageBuilder()
// 6 partitions the metadata image.
.addTopic(fooTopicId, fooTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
))))
.build();
@@ -3405,8 +3410,8 @@ public void testOnNewMetadataImageWithEmptyDelta() {
MetadataDelta delta = new MetadataDelta(MetadataImage.EMPTY);
MetadataImage image = delta.apply(MetadataProvenance.EMPTY);
- context.groupMetadataManager.onNewMetadataImage(image, delta);
- assertEquals(image, context.groupMetadataManager.image());
+ context.groupMetadataManager.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(delta));
+ assertEquals(new KRaftCoordinatorMetadataImage(image), context.groupMetadataManager.image());
}
@Test
@@ -3479,12 +3484,12 @@ public void testOnNewMetadataImage() {
image = delta.apply(MetadataProvenance.EMPTY);
// Update metadata image with the delta.
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(delta));
// Verify the groups.
List.of("group1", "group2", "group3", "group4").forEach(groupId -> {
ConsumerGroup group = context.groupMetadataManager.consumerGroup(groupId);
- assertTrue(group.hasMetadataExpired(context.time.milliseconds()));
+ assertTrue(group.hasMetadataExpired(context.time.milliseconds()), groupId);
});
List.of("group5").forEach(groupId -> {
@@ -3493,7 +3498,7 @@ public void testOnNewMetadataImage() {
});
// Verify image.
- assertEquals(image, context.groupMetadataManager.image());
+ assertEquals(new KRaftCoordinatorMetadataImage(image), context.groupMetadataManager.image());
}
@Test
@@ -3508,10 +3513,10 @@ public void testSessionTimeoutLifecycle() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .build()))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -3583,10 +3588,10 @@ public void testSessionTimeoutExpiration() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .build()))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -3646,7 +3651,7 @@ public void testOnLoadedSessionTimeoutExpiration() {
.build();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder("foo-1")
.setState(MemberState.STABLE)
@@ -3663,7 +3668,7 @@ public void testOnLoadedSessionTimeoutExpiration() {
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage))
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage)))
)))
.build();
@@ -3711,10 +3716,10 @@ public void testSessionTimeoutExpirationForShareMember() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .build()))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -3771,7 +3776,7 @@ public void testOnLoadedSessionTimeoutExpirationForShareMember() {
.build();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withShareGroup(new ShareGroupBuilder(groupId, 10)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -3787,7 +3792,7 @@ public void testOnLoadedSessionTimeoutExpirationForShareMember() {
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
))))
.build();
@@ -3835,10 +3840,10 @@ public void testSessionTimeoutExpirationStaticMember() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .build()))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -3916,10 +3921,10 @@ public void testRebalanceTimeoutLifecycle() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 3)
.addRacks()
- .build())
+ .build()))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(Map.of(memberId1, new MemberAssignmentImpl(mkAssignment(
@@ -4066,7 +4071,7 @@ public void testRebalanceTimeoutExpiration() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -4170,7 +4175,7 @@ memberId2, new MemberAssignmentImpl(mkAssignment(
GroupCoordinatorRecordHelpers.newConsumerGroupTargetAssignmentTombstoneRecord(groupId, memberId1),
GroupCoordinatorRecordHelpers.newConsumerGroupMemberSubscriptionTombstoneRecord(groupId, memberId1),
GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 3, computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
)))
)
)
@@ -4195,7 +4200,7 @@ public void testOnLoaded() {
.addTopic(barTopicId, barTopicName, 3)
.build();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withConsumerGroup(new ConsumerGroupBuilder("foo", 10)
.withMember(new ConsumerGroupMember.Builder("foo-1")
.setState(MemberState.UNREVOKED_PARTITIONS)
@@ -4223,7 +4228,7 @@ public void testOnLoaded() {
mkTopicAssignment(fooTopicId, 3, 4, 5)))
.withAssignmentEpoch(10)
.withMetadataHash(computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, metadataImage)
+ fooTopicName, computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(metadataImage))
))))
.build();
@@ -4247,10 +4252,10 @@ public void testOnLoadedWithStreamsGroup() {
String barTopicName = "bar";
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build())
+ .build()))
.withStreamsGroup(new StreamsGroupBuilder("foo", 10)
.withMember(new StreamsGroupMember.Builder("foo-1")
.setState(org.apache.kafka.coordinator.group.streams.MemberState.UNREVOKED_TASKS)
@@ -4313,10 +4318,10 @@ public void testOnLoadedWithShareGroup() {
String barTopicName = "bar";
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(new MetadataImageBuilder()
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build())
+ .build()))
.withShareGroup(new ShareGroupBuilder("foo", 10)
.withMember(new ShareGroupMember.Builder("foo-1")
.setState(MemberState.STABLE)
@@ -9538,7 +9543,7 @@ public void testConsumerGroupDescribeNoErrors() {
.setMembers(List.of(
memberBuilder.build().asConsumerGroupDescribeMember(
new Assignment(Map.of()),
- new MetadataImageBuilder().build().topics()
+ new MetadataImageBuilder().buildCoordinatorMetadataImage()
)
))
.setGroupState(ConsumerGroup.ConsumerGroupState.ASSIGNING.toString())
@@ -9578,9 +9583,9 @@ public void testConsumerGroupDescribeBeforeAndAfterCommittingOffset() {
String memberId2 = "memberId2";
String topicName = "topicName";
Uuid topicId = Uuid.randomUuid();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(topicId, topicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -9618,8 +9623,8 @@ public void testConsumerGroupDescribeBeforeAndAfterCommittingOffset() {
describedGroup = new ConsumerGroupDescribeResponseData.DescribedGroup()
.setGroupId(consumerGroupId)
.setMembers(List.of(
- memberBuilder1.build().asConsumerGroupDescribeMember(new Assignment(Map.of()), metadataImage.topics()),
- memberBuilder2.build().asConsumerGroupDescribeMember(new Assignment(assignmentMap), metadataImage.topics())
+ memberBuilder1.build().asConsumerGroupDescribeMember(new Assignment(Map.of()), metadataImage),
+ memberBuilder2.build().asConsumerGroupDescribeMember(new Assignment(assignmentMap), metadataImage)
))
.setGroupState(ConsumerGroup.ConsumerGroupState.ASSIGNING.toString())
.setAssignorName("range")
@@ -10493,7 +10498,7 @@ public void testConsumerGroupRebalanceSensor() {
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -10647,11 +10652,11 @@ memberId2, new MemberAssignmentImpl(mkAssignment(
))
)));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -10808,11 +10813,11 @@ memberId3, new MemberAssignmentImpl(mkAssignment(
))
)));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -11038,11 +11043,11 @@ memberId2, new MemberAssignmentImpl(mkAssignment(
))
)));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -11124,10 +11129,10 @@ public void testConsumerGroupHeartbeatToClassicGroupFromExistingStaticMember() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -11284,11 +11289,11 @@ public void testConsumerGroupHeartbeatToClassicGroupWithEmptyAssignmentMember()
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -11458,11 +11463,11 @@ public void testConsumerGroupHeartbeatFromExistingClassicStaticMember() {
mkTopicAssignment(barTopicId, 2)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with two static members.
// Member 1 uses the classic protocol and member 2 uses the consumer protocol.
@@ -11575,11 +11580,11 @@ memberId3, new MemberAssignmentImpl(mkAssignment(
))
)));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_MIGRATION_POLICY_CONFIG, ConsumerGroupMigrationPolicy.UPGRADE.toString())
@@ -11953,11 +11958,11 @@ public void testLastConsumerProtocolMemberLeavingConsumerGroup() {
mkTopicAssignment(barTopicId, 2)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with two members.
// Member 1 uses the classic protocol and member 2 uses the consumer protocol.
@@ -12133,11 +12138,11 @@ public void testLastConsumerProtocolMemberSessionTimeoutInConsumerGroup() {
mkTopicAssignment(barTopicId, 2)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with two members.
// Member 1 uses the classic protocol and member 2 uses the consumer protocol.
@@ -12313,12 +12318,12 @@ public void testLastConsumerProtocolMemberRebalanceTimeoutInConsumerGroup() {
mkTopicAssignment(barTopicId, 2)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with two members.
// Member 1 uses the classic protocol and member 2 uses the consumer protocol.
@@ -12514,11 +12519,11 @@ public void testLastStaticConsumerProtocolMemberReplacedByClassicProtocolMember(
mkTopicAssignment(fooTopicId, 3, 4, 5)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 2)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with two members.
// Member 1 uses the classic protocol and static member 2 uses the consumer protocol.
@@ -12754,11 +12759,11 @@ public void testJoiningConsumerGroupWithNewDynamicMember() throws Exception {
String memberId = Uuid.randomUuid().toString();
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -12897,10 +12902,10 @@ memberId, new MemberAssignmentImpl(mkAssignment(
))
)));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(metadataImage)
@@ -12951,11 +12956,11 @@ public void testJoiningConsumerGroupWithNewStaticMember() throws Exception {
String memberId = Uuid.randomUuid().toString();
String instanceId = "instance-id";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -13042,10 +13047,10 @@ public void testJoiningConsumerGroupReplacingExistingStaticMember() throws Excep
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
String memberId = Uuid.randomUuid().toString();
String instanceId = "instance-id";
@@ -13164,12 +13169,12 @@ public void testJoiningConsumerGroupWithExistingStaticMemberAndNewSubscription()
String memberId2 = Uuid.randomUuid().toString();
String instanceId = "instance-id";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage),
barTopicName, computeTopicHash(barTopicName, metadataImage),
@@ -13393,12 +13398,12 @@ public void testReconciliationInJoiningConsumerGroupWithEagerProtocol() throws E
String memberId1 = Uuid.randomUuid().toString();
String memberId2 = Uuid.randomUuid().toString();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
long zarTopicHash = computeTopicHash(zarTopicName, metadataImage);
@@ -13632,12 +13637,12 @@ public void testReconciliationInJoiningConsumerGroupWithCooperativeProtocol() th
String memberId1 = Uuid.randomUuid().toString();
String memberId2 = Uuid.randomUuid().toString();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
long zarTopicHash = computeTopicHash(zarTopicName, metadataImage);
@@ -14020,7 +14025,7 @@ public void testClassicGroupSyncToConsumerGroupWithAllConsumerProtocolVersions()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(member1)
.withMember(member2)
@@ -14708,11 +14713,11 @@ public void testConsumerGroupMemberUsingClassicProtocolBatchLeaveGroup() {
.setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with three members.
// Dynamic member 1 uses the classic protocol.
@@ -14892,11 +14897,11 @@ public void testConsumerGroupMemberUsingClassicProtocolBatchLeaveGroupUpdatingSu
.setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 0)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -15079,11 +15084,11 @@ public void testLastConsumerProtocolMemberLeavingConsumerGroupByAdminApi() {
.setAssignedPartitions(mkAssignment(mkTopicAssignment(barTopicId, 1)))
.build();
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
.addTopic(barTopicId, barTopicName, 2)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
// Consumer group with four members.
// Dynamic member 1 uses the classic protocol.
@@ -15303,15 +15308,11 @@ public void testShareGroupDescribeNoErrors() {
Uuid topicId = Uuid.randomUuid();
String topicName = "foo";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId, topicName, 1)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta.Builder()
- .setImage(image)
- .build();
-
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
new ShareGroupHeartbeatRequestData()
@@ -15354,7 +15355,7 @@ public void testShareGroupDescribeNoErrors() {
.setSubscribedTopicNames(List.of(topicName))
.build()
.asShareGroupDescribeMember(
- new MetadataImageBuilder().build().topics()
+ new MetadataImageBuilder().buildCoordinatorMetadataImage()
)
))
.setGroupState(ShareGroup.ShareGroupState.STABLE.toString())
@@ -15370,7 +15371,7 @@ public void testShareGroupMemberIdGeneration() {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -15384,16 +15385,12 @@ public void testShareGroupMemberIdGeneration() {
String topicName2 = "bar";
String groupId = "group-foo";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 1)
.addTopic(topicId2, topicName2, 1)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta.Builder()
- .setImage(image)
- .build();
-
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
new ShareGroupHeartbeatRequestData()
@@ -15467,16 +15464,12 @@ public void testShareGroupUnknownMemberIdJoins() {
Uuid topicId2 = Uuid.randomUuid();
String topicName2 = "bar";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 1)
.addTopic(topicId2, topicName2, 1)
- .build();
-
- MetadataDelta delta = new MetadataDelta.Builder()
- .setImage(image)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
// A first member joins to create the group.
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
@@ -15527,7 +15520,7 @@ public void testShareGroupMemberJoinsEmptyGroupWithAssignments() {
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -15544,12 +15537,14 @@ public void testShareGroupMemberJoinsEmptyGroupWithAssignments() {
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.build();
+
+ CoordinatorMetadataImage coordinatorMetadataImage = new KRaftCoordinatorMetadataImage(image);
MetadataDelta delta = new MetadataDelta.Builder()
.setImage(image)
.build();
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(coordinatorMetadataImage, new KRaftCoordinatorMetadataDelta(delta));
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
new ShareGroupHeartbeatRequestData()
@@ -15603,8 +15598,8 @@ public void testShareGroupMemberJoinsEmptyGroupWithAssignments() {
List expectedRecords = List.of(
GroupCoordinatorRecordHelpers.newShareGroupMemberSubscriptionRecord(groupId, expectedMember),
GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 1, computeGroupHash(Map.of(
- fooTopicName, computeTopicHash(fooTopicName, image),
- barTopicName, computeTopicHash(barTopicName, image)
+ fooTopicName, computeTopicHash(fooTopicName, coordinatorMetadataImage),
+ barTopicName, computeTopicHash(barTopicName, coordinatorMetadataImage)
))),
GroupCoordinatorRecordHelpers.newShareGroupTargetAssignmentRecord(groupId, memberId, mkAssignment(
mkTopicAssignment(fooTopicId, 0, 1, 2, 3, 4, 5),
@@ -15653,12 +15648,12 @@ public void testShareGroupLeavingMemberBumpsGroupEpoch() {
Uuid zarTopicId = Uuid.randomUuid();
String zarTopicName = "zar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.addTopic(zarTopicId, zarTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -15765,16 +15760,12 @@ public void testShareGroupNewMemberIsRejectedWithMaximumMembersIsReached() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addTopic(barTopicId, barTopicName, 1)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta.Builder()
- .setImage(image)
- .build();
-
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
// Member 1 joins the group.
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
@@ -15952,7 +15943,7 @@ public void testStreamsGroupMemberEpochValidation() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(Map.of(memberId, TasksTuple.EMPTY));
@@ -16044,7 +16035,7 @@ public void testStreamsOwnedTasksValidation() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.withStreamsGroup(new StreamsGroupBuilder(groupId, 10)
.withMember(streamsGroupMemberBuilderWithDefaults(memberId)
.setMemberEpoch(10)
@@ -16098,7 +16089,7 @@ public void testStreamsNewMemberIsRejectedWithMaximumMembersIsReached() {
// Create a context with one streams group containing two members.
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(new MetadataImageBuilder().build())
+ .withMetadataImage(new MetadataImageBuilder().buildCoordinatorMetadataImage())
.withConfig(GroupCoordinatorConfig.STREAMS_GROUP_MAX_SIZE_CONFIG, 2)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 10)
.withMember(streamsGroupMemberBuilderWithDefaults(memberId1)
@@ -16146,10 +16137,10 @@ public void testMemberJoinsEmptyStreamsGroup() {
));
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage),
barTopicName, computeTopicHash(barTopicName, metadataImage)
@@ -16239,9 +16230,9 @@ public void testStreamsGroupMemberJoiningWithMissingSourceTopic() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -16320,9 +16311,9 @@ public void testStreamsGroupMemberJoiningWithMissingInternalTopic() {
)
);
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -16407,10 +16398,10 @@ public void testStreamsGroupMemberJoiningWithIncorrectlyPartitionedTopic() {
)
);
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withStreamsGroupTaskAssignors(List.of(assignor))
@@ -16495,10 +16486,10 @@ public void testStreamsGroupMemberJoiningWithStaleTopology() {
)
);
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withStreamsGroupTaskAssignors(List.of(assignor))
@@ -16580,9 +16571,9 @@ public void testStreamsGroupMemberRequestingShutdownApplication() {
new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
long groupMetadataHash = computeGroupHash(Map.of(fooTopicName, computeTopicHash(fooTopicName, metadataImage)));
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
@@ -16675,9 +16666,9 @@ public void testStreamsGroupMemberRequestingShutdownApplicationUponLeaving() {
new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -16770,10 +16761,11 @@ public void testStreamsUpdatingMemberMetadataTriggersNewTargetAssignment() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
+
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage),
barTopicName, computeTopicHash(barTopicName, metadataImage)
@@ -16873,15 +16865,15 @@ public void testStreamsUpdatingPartitionMetadataTriggersNewTargetAssignment() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage newMetadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage newMetadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, changedPartitionCount)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataImage oldMetadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage oldMetadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
long oldGroupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, oldMetadataImage),
barTopicName, computeTopicHash(barTopicName, oldMetadataImage)
@@ -16983,10 +16975,10 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -16994,7 +16986,7 @@ public void testStreamsNewJoiningMemberTriggersNewTargetAssignment() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.withStreamsGroup(new StreamsGroupBuilder(groupId, 10)
.withMember(streamsGroupMemberBuilderWithDefaults(memberId1)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -17087,7 +17079,7 @@ public void testStreamsLeavingMemberRemovesMemberAndBumpsGroupEpoch() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.withStreamsGroup(new StreamsGroupBuilder(groupId, 10)
.withMember(streamsGroupMemberBuilderWithDefaults(memberId1)
.setMemberEpoch(10)
@@ -17156,7 +17148,7 @@ public void testStreamsGroupHeartbeatPartialResponseWhenNothingChanges() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Prepare new assignment for the group.
@@ -17225,10 +17217,10 @@ public void testStreamsReconciliationProcess() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage),
barTopicName, computeTopicHash(barTopicName, metadataImage)
@@ -17682,10 +17674,11 @@ public void testStreamsStreamsGroupStates() {
new Subtopology().setSubtopologyId(subtopology2).setSourceTopics(List.of(barTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build();
+ .buildCoordinatorMetadataImage();
+
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage),
barTopicName, computeTopicHash(barTopicName, metadataImage)
@@ -17712,7 +17705,7 @@ barTopicName, computeTopicHash(barTopicName, metadataImage)
new LogContext(),
groupMetadataHash,
StreamsTopology.fromRecord(StreamsCoordinatorRecordHelpers.convertToStreamsGroupTopologyRecord(topology)),
- metadataImage.topics()));
+ metadataImage));
assertEquals(StreamsGroup.StreamsGroupState.ASSIGNING, context.streamsGroupState(groupId));
@@ -17768,7 +17761,7 @@ public void testStreamsTaskAssignorExceptionOnRegularHeartbeat() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Member 1 joins the streams group. The request fails because the
@@ -17798,9 +17791,9 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() {
new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -17820,7 +17813,11 @@ public void testStreamsPartitionMetadataRefreshedAfterGroupIsLoaded() {
.withMetadataHash(computeGroupHash(Map.of(
// foo only has 3 tasks stored in the metadata but foo has
// 6 partitions the metadata image.
- fooTopicName, computeTopicHash(fooTopicName, new MetadataImageBuilder().addTopic(fooTopicId, fooTopicName, 3).build())
+ fooTopicName, computeTopicHash(
+ fooTopicName,
+ new MetadataImageBuilder()
+ .addTopic(fooTopicId, fooTopicName, 3)
+ .buildCoordinatorMetadataImage())
))))
.build();
@@ -17895,9 +17892,9 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() {
new Subtopology().setSubtopologyId(subtopology1).setSourceTopics(List.of(fooTopicName))
));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
MockTaskAssignor assignor = new MockTaskAssignor("sticky");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -17917,7 +17914,11 @@ public void testStreamsPartitionMetadataRefreshedAgainAfterWriteFailure() {
.withMetadataHash(computeGroupHash(Map.of(
// foo only has 3 partitions stored in the metadata but foo has
// 6 partitions the metadata image.
- fooTopicName, computeTopicHash(fooTopicName, new MetadataImageBuilder().addTopic(fooTopicId, fooTopicName, 3).build())
+ fooTopicName, computeTopicHash(
+ fooTopicName,
+ new MetadataImageBuilder()
+ .addTopic(fooTopicId, fooTopicName, 3)
+ .buildCoordinatorMetadataImage())
))))
.build();
@@ -18018,7 +18019,7 @@ public void testStreamsSessionTimeoutLifecycle() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE,
@@ -18094,7 +18095,7 @@ public void testStreamsSessionTimeoutExpiration() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(Map.of(memberId, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE,
@@ -18159,7 +18160,7 @@ public void testStreamsRebalanceTimeoutLifecycle() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(Map.of(memberId1, TaskAssignmentTestUtil.mkTasksTuple(TaskRole.ACTIVE,
@@ -18318,7 +18319,7 @@ public void testStreamsRebalanceTimeoutExpiration() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 3)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(
@@ -18523,7 +18524,7 @@ public void testStreamsOnNewMetadataImage() {
image = delta.apply(MetadataProvenance.EMPTY);
// Update metadata image with the delta.
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), new KRaftCoordinatorMetadataDelta(delta));
// Verify the groups.
List.of("group1", "group2", "group3", "group4").forEach(groupId -> {
@@ -18537,7 +18538,7 @@ public void testStreamsOnNewMetadataImage() {
});
// Verify image.
- assertEquals(image, context.groupMetadataManager.image());
+ assertEquals(new KRaftCoordinatorMetadataImage(image), context.groupMetadataManager.image());
}
@Test
@@ -18556,7 +18557,7 @@ public void testStreamsGroupEndpointInformationOnlyWhenEpochGreater() {
.withStreamsGroupTaskAssignors(List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Prepare new assignment for the group.
@@ -18647,7 +18648,7 @@ public void testConsumerGroupDynamicConfigs() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -18734,7 +18735,7 @@ public void testShareGroupDynamicConfigs() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(new GroupAssignment(
@@ -18743,15 +18744,11 @@ public void testShareGroupDynamicConfigs() {
)))
));
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta.Builder()
- .setImage(image)
- .build();
-
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
// Session timer is scheduled on first heartbeat.
CoordinatorResult>, CoordinatorRecord> result =
@@ -18871,7 +18868,7 @@ public void testStreamsGroupDynamicConfigs() {
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addRacks()
- .build())
+ .buildCoordinatorMetadataImage())
.build();
assignor.prepareGroupAssignment(
@@ -19512,7 +19509,7 @@ public void testConsumerGroupHeartbeatOnShareGroup() {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withShareGroup(new ShareGroupBuilder(groupId, 1)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -19545,7 +19542,7 @@ public void testClassicGroupJoinOnShareGroup() throws Exception {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withShareGroup(new ShareGroupBuilder(groupId, 1)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -19579,7 +19576,7 @@ public void testClassicGroupSyncOnShareGroup() throws Exception {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withShareGroup(new ShareGroupBuilder(groupId, 1)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -19614,7 +19611,7 @@ public void testClassicGroupLeaveOnShareGroup() throws Exception {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withShareGroup(new ShareGroupBuilder(groupId, 1)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -19644,7 +19641,7 @@ public void testConsumerGroupDescribeOnShareGroup() {
MockPartitionAssignor assignor = new MockPartitionAssignor("share");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withShareGroupAssignor(assignor)
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withShareGroup(new ShareGroupBuilder(groupId, 1)
.withMember(new ShareGroupMember.Builder(memberId)
.setState(MemberState.STABLE)
@@ -19685,7 +19682,7 @@ public void testShareGroupHeartbeatOnConsumerGroup() {
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -19749,7 +19746,7 @@ public void testConsumerGroupHeartbeatOnStreamsGroup() {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19779,7 +19776,7 @@ public void testShareGroupHeartbeatOnStreamsGroup() {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19807,7 +19804,7 @@ public void testClassicGroupJoinOnStreamsGroup() throws Exception {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19838,7 +19835,7 @@ public void testClassicGroupSyncOnStreamsGroup() throws Exception {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19870,7 +19867,7 @@ public void testClassicGroupLeaveOnStreamsGroup() throws Exception {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19897,7 +19894,7 @@ public void testConsumerGroupDescribeOnStreamsGroup() {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19927,7 +19924,7 @@ public void testShareGroupDescribeOnStreamsGroup() {
String memberId = Uuid.randomUuid().toString();
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
- .withMetadataImage(MetadataImage.EMPTY)
+ .withMetadataImage(CoordinatorMetadataImage.EMPTY)
.withStreamsGroup(new StreamsGroupBuilder(groupId, 1)
.withMember(StreamsGroupMember.Builder.withDefaults(memberId)
.setState(org.apache.kafka.coordinator.group.streams.MemberState.STABLE)
@@ -19967,7 +19964,7 @@ public void testStreamsGroupHeartbeatOnConsumerGroup() {
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -20036,7 +20033,7 @@ public void testStreamsGroupHeartbeatOnShareGroup() {
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build())
+ .buildCoordinatorMetadataImage())
.withShareGroup(new ShareGroupBuilder(groupId, 10)
.withMember(new ShareGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -20195,7 +20192,7 @@ memberId2, new MemberAssignmentImpl(mkAssignment(
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
.withMetadataImage(new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 2)
- .build())
+ .buildCoordinatorMetadataImage())
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -20245,9 +20242,9 @@ public void testConsumerGroupMemberJoinsWithNewRegex() {
Uuid fooTopicId = Uuid.randomUuid();
String fooTopicName = "foo";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
- .build(12345L);
+ .buildCoordinatorMetadataImage(12345L);
long groupMetadataHash = computeGroupHash(Map.of(
fooTopicName, computeTopicHash(fooTopicName, metadataImage)
));
@@ -20349,10 +20346,10 @@ public void testConsumerGroupMemberJoinsWithUpdatedRegex() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(12345L);
+ .buildCoordinatorMetadataImage(12345L);
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
@@ -20463,10 +20460,10 @@ public void testConsumerGroupMemberJoinsWithRegexAndUpdatesItBeforeResolutionCom
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
assignor.prepareGroupAssignment(new GroupAssignment(Map.of()));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(12345L);
+ .buildCoordinatorMetadataImage(12345L);
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
@@ -20637,12 +20634,12 @@ public void testConsumerGroupMemberJoinRefreshesExpiredRegexes() {
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
.build(1L);
- long fooTopicHash = computeTopicHash(fooTopicName, image);
- long barTopicHash = computeTopicHash(barTopicName, image);
+ long fooTopicHash = computeTopicHash(fooTopicName, new KRaftCoordinatorMetadataImage(image));
+ long barTopicHash = computeTopicHash(barTopicName, new KRaftCoordinatorMetadataImage(image));
GroupMetadataManagerTestContext context = new GroupMetadataManagerTestContext.Builder()
.withConfig(GroupCoordinatorConfig.CONSUMER_GROUP_ASSIGNORS_CONFIG, List.of(assignor))
- .withMetadataImage(image)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(image))
.withConsumerGroup(new ConsumerGroupBuilder(groupId, 10)
.withMember(new ConsumerGroupMember.Builder(memberId1)
.setState(MemberState.STABLE)
@@ -20691,8 +20688,8 @@ public void testConsumerGroupMemberJoinRefreshesExpiredRegexes() {
.build(2L);
context.groupMetadataManager.onNewMetadataImage(
- newImage,
- new MetadataDelta(newImage)
+ new KRaftCoordinatorMetadataImage(newImage),
+ new KRaftCoordinatorMetadataDelta(new MetadataDelta(newImage))
);
// A member heartbeats.
@@ -20751,7 +20748,7 @@ public void testConsumerGroupMemberJoinRefreshesExpiredRegexes() {
List.of(GroupCoordinatorRecordHelpers.newConsumerGroupEpochRecord(groupId, 11, computeGroupHash(Map.of(
fooTopicName, fooTopicHash,
barTopicName, barTopicHash,
- foooTopicName, computeTopicHash(foooTopicName, newImage)
+ foooTopicName, computeTopicHash(foooTopicName, new KRaftCoordinatorMetadataImage(newImage))
))))
),
task.result.records()
@@ -20769,10 +20766,10 @@ public void testConsumerGroupMemberJoinsWithRegexWithTopicAuthorizationFailure()
String fooTopicName = "foo";
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(12345L);
+ .buildCoordinatorMetadataImage(12345L);
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -20997,10 +20994,10 @@ public void testConsumerGroupMemberJoinsRefreshTopicAuthorization() {
String fooTopicName = "foo";
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(12345L);
+ .buildCoordinatorMetadataImage(12345L);
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -21227,10 +21224,10 @@ public void testResolvedRegularExpressionsRemovedWhenMembersLeaveOrFenced() {
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
assignor.prepareGroupAssignment(new GroupAssignment(Map.of()));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(1L);
+ .buildCoordinatorMetadataImage(1L);
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -21344,10 +21341,10 @@ public void testResolvedRegularExpressionsRemovedWhenConsumerMembersRemovedByAdm
MockPartitionAssignor assignor = new MockPartitionAssignor("range");
assignor.prepareGroupAssignment(new GroupAssignment(Map.of()));
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 6)
.addTopic(barTopicId, barTopicName, 3)
- .build(1L);
+ .buildCoordinatorMetadataImage(1L);
long fooTopicHash = computeTopicHash(fooTopicName, metadataImage);
long barTopicHash = computeTopicHash(barTopicName, metadataImage);
@@ -21500,13 +21497,12 @@ public void testShareGroupDeleteRequestNoDeletingTopics() {
when(shareGroup.groupId()).thenReturn(groupId);
when(shareGroup.isEmpty()).thenReturn(false);
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, t1Name, 2)
.addTopic(t2Uuid, t2Name, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta(image);
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21566,14 +21562,13 @@ public void testShareGroupDeleteRequestWithAlreadyDeletingTopics() {
when(shareGroup.groupId()).thenReturn(groupId);
when(shareGroup.isEmpty()).thenReturn(false);
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, t1Name, 2)
.addTopic(t2Uuid, t2Name, 2)
.addTopic(t3Uuid, t3Name, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta(image);
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21634,13 +21629,13 @@ public void testShareGroupDeleteRequestWithAlreadyDeletingTopicsButNotInMetadata
when(shareGroup.groupId()).thenReturn(groupId);
when(shareGroup.isEmpty()).thenReturn(false);
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, t1Name, 2)
.addTopic(t2Uuid, t2Name, 2)
// .addTopic(t3Uuid, t3Name, 2) // Simulate deleting topic not present in metadata image.
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta(image);
+ CoordinatorMetadataDelta delta = image.emptyDelta();
context.groupMetadataManager.onNewMetadataImage(image, delta);
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21701,8 +21696,8 @@ public void testShareGroupDeleteRequestWithAlreadyDeletingTopicsButMetadataIsEmp
when(shareGroup.groupId()).thenReturn(groupId);
when(shareGroup.isEmpty()).thenReturn(false);
- MetadataImage image = MetadataImage.EMPTY;
- MetadataDelta delta = new MetadataDelta(image);
+ CoordinatorMetadataImage image = CoordinatorMetadataImage.EMPTY;
+ CoordinatorMetadataDelta delta = image.emptyDelta();
context.groupMetadataManager.onNewMetadataImage(image, delta);
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21757,12 +21752,12 @@ public void testSharePartitionsEligibleForOffsetDeletionSuccess() {
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21848,14 +21843,14 @@ public void testSharePartitionsEligibleForOffsetDeletionContainsDeletingTopics()
Uuid topicId3 = Uuid.randomUuid();
Uuid topicId4 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
.addTopic(topicId3, topicName3, 2)
.addTopic(topicId4, topicName4, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -21958,11 +21953,11 @@ public void testSharePartitionsEligibleForOffsetDeletionErrorTopics() {
String topicName2 = "topic-2";
Uuid topicId1 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -22041,12 +22036,12 @@ public void testSharePartitionsEligibleForOffsetDeletionUninitializedTopics() {
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -22126,12 +22121,12 @@ public void testSharePartitionsEligibleForOffsetDeletionUninitializedAndErrorTop
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -22216,12 +22211,12 @@ public void testCompleteDeleteShareGroupOffsets() {
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -22289,12 +22284,12 @@ public void testCompleteDeleteShareGroupOffsetsEmptyResult() {
Uuid topicId1 = Uuid.randomUuid();
Uuid topicId2 = Uuid.randomUuid();
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(topicId1, topicName1, 3)
.addTopic(topicId2, topicName2, 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.replay(GroupCoordinatorRecordHelpers.newShareGroupEpochRecord(groupId, 0, 0));
@@ -22328,14 +22323,14 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() {
String t1Name = "t1";
Uuid t2Uuid = Uuid.randomUuid();
String t2Name = "t2";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, "t1", 2)
.addTopic(t2Uuid, "t2", 2)
- .build();
+ .buildCoordinatorMetadataImage();
String groupId = "share-group";
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
Uuid memberId = Uuid.randomUuid();
CoordinatorResult>, CoordinatorRecord> result = context.shareGroupHeartbeat(
@@ -22395,9 +22390,9 @@ public void testShareGroupHeartbeatInitializeOnPartitionUpdate() {
image = new MetadataImageBuilder()
.addTopic(t1Uuid, "t1", 4)
.addTopic(t2Uuid, "t2", 2)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
assignor.prepareGroupAssignment(new GroupAssignment(
Map.of(
@@ -22467,13 +22462,13 @@ public void testShareGroupHeartbeatPersisterRequestWithInitializing() {
Uuid t1Uuid = Uuid.randomUuid();
String t1Name = "t1";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, t1Name, 2)
- .build();
+ .buildCoordinatorMetadataImage();
String groupId = "share-group";
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.groupMetadataManager.replay(
new ShareGroupMetadataKey()
.setGroupId(groupId),
@@ -22573,13 +22568,13 @@ public void testShareGroupInitializingClearsCommonDeleting() {
Uuid t1Uuid = Uuid.randomUuid();
String t1Name = "t1";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Uuid, t1Name, 2)
- .build();
+ .buildCoordinatorMetadataImage();
String groupId = "share-group";
- context.groupMetadataManager.onNewMetadataImage(image, mock(MetadataDelta.class));
+ context.groupMetadataManager.onNewMetadataImage(image, mock(CoordinatorMetadataDelta.class));
context.groupMetadataManager.replay(
new ShareGroupMetadataKey()
.setGroupId(groupId),
@@ -22638,7 +22633,7 @@ public void testShareGroupInitializeSuccess() {
.withShareGroupAssignor(assignor)
.withMetadataImage(new MetadataImageBuilder()
.addTopic(topicId, topicName, 2)
- .build()
+ .buildCoordinatorMetadataImage()
)
.build();
@@ -22735,7 +22730,7 @@ public void testSubscribedTopicsChangeMap() {
.withConfig(GroupCoordinatorConfig.SHARE_GROUP_INITIALIZE_RETRY_INTERVAL_MS_CONFIG, initRetryTimeoutMs)
.withMetadataImage(new MetadataImageBuilder()
.addTopic(topicId, topicName, partitions)
- .build())
+ .buildCoordinatorMetadataImage())
.build();
// Empty on empty subscription topics
@@ -22789,13 +22784,13 @@ topicId, new InitMapValue(topicName, Set.of(0), timeNow)
.setDeletingTopics(List.of())
);
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(t1Id, t1Name, 2)
.addTopic(t2Id, t2Name, 2)
.addTopic(t3Id, t3Name, 3)
- .build();
+ .buildCoordinatorMetadataImage();
- context.groupMetadataManager.onNewMetadataImage(metadataImage, new MetadataDelta(metadataImage));
+ context.groupMetadataManager.onNewMetadataImage(metadataImage, metadataImage.emptyDelta());
// Since t1 is initializing and t2 is initialized due to replay above.
timeNow = timeNow + initRetryTimeoutMs + 1;
@@ -22836,13 +22831,12 @@ public void testUninitializeTopics() {
result.records()
);
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Id, t1Name, 2)
.addTopic(t2Id, t2Name, 3)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta(image);
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
// Cleanup happens from initialzing state only.
context.groupMetadataManager.replay(
@@ -22918,17 +22912,16 @@ public void testMaybeCleanupShareGroupStateInitDeletedTopicsPresent() {
Uuid t6Id = Uuid.randomUuid();
String t6Name = "t6";
- MetadataImage image = new MetadataImageBuilder()
+ CoordinatorMetadataImage image = new MetadataImageBuilder()
.addTopic(t1Id, t1Name, 2)
.addTopic(t2Id, t2Name, 3)
.addTopic(t3Id, t3Name, 3)
.addTopic(t4Id, t4Name, 3)
.addTopic(t5Id, t5Name, 3)
.addTopic(t6Id, t6Name, 3)
- .build();
+ .buildCoordinatorMetadataImage();
- MetadataDelta delta = new MetadataDelta(image);
- context.groupMetadataManager.onNewMetadataImage(image, delta);
+ context.groupMetadataManager.onNewMetadataImage(image, image.emptyDelta());
context.groupMetadataManager.replay(
new ShareGroupMetadataKey()
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
index bc88675ea9880..cf1ff5350cdf8 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/GroupMetadataManagerTestContext.java
@@ -53,6 +53,7 @@
import org.apache.kafka.common.security.auth.SecurityProtocol;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorExecutor;
@@ -115,7 +116,6 @@
import org.apache.kafka.coordinator.group.streams.TasksTuple;
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor;
import org.apache.kafka.coordinator.group.streams.topics.InternalTopicManager;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.authorizer.Authorizer;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.share.persister.InitializeShareGroupStateParameters;
@@ -462,7 +462,7 @@ public static class Builder {
private final MockCoordinatorExecutor executor = new MockCoordinatorExecutor<>();
private final LogContext logContext = new LogContext();
private final SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
private GroupConfigManager groupConfigManager;
private final List consumerGroupBuilders = new ArrayList<>();
private final List streamsGroupBuilders = new ArrayList<>();
@@ -478,7 +478,7 @@ public Builder withConfig(String key, Object value) {
return this;
}
- public Builder withMetadataImage(MetadataImage metadataImage) {
+ public Builder withMetadataImage(CoordinatorMetadataImage metadataImage) {
this.metadataImage = metadataImage;
return this;
}
@@ -519,7 +519,7 @@ public Builder withTime(MockTime time) {
}
public GroupMetadataManagerTestContext build() {
- if (metadataImage == null) metadataImage = MetadataImage.EMPTY;
+ if (metadataImage == null) metadataImage = CoordinatorMetadataImage.EMPTY;
if (groupConfigManager == null) groupConfigManager = createConfigManager();
config.putIfAbsent(
@@ -554,7 +554,7 @@ public GroupMetadataManagerTestContext build() {
);
consumerGroupBuilders.forEach(builder -> builder.build().forEach(context::replay));
- shareGroupBuilders.forEach(builder -> builder.build(metadataImage.topics()).forEach(context::replay));
+ shareGroupBuilders.forEach(builder -> builder.build().forEach(context::replay));
streamsGroupBuilders.forEach(builder -> {
builder.build().forEach(context::replay);
StreamsGroup group = context.groupMetadataManager.getStreamsGroupOrThrow(builder.groupId());
@@ -563,7 +563,7 @@ public GroupMetadataManagerTestContext build() {
new LogContext(),
0,
group.topology().get(),
- metadataImage.topics())
+ metadataImage)
);
}
});
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
index 6ab8cd4cbddc6..fc72a394c7ac3 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/OffsetMetadataManagerTest.java
@@ -49,6 +49,7 @@
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorExecutor;
import org.apache.kafka.coordinator.common.runtime.MockCoordinatorTimer;
import org.apache.kafka.coordinator.group.classic.ClassicGroup;
@@ -140,7 +141,7 @@ OffsetMetadataManagerTestContext build() {
.withExecutor(executor)
.withSnapshotRegistry(snapshotRegistry)
.withLogContext(logContext)
- .withMetadataImage(metadataImage)
+ .withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withGroupCoordinatorMetricsShard(metrics)
.withGroupConfigManager(configManager)
.withConfig(GroupCoordinatorConfig.fromProps(Map.of()))
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/UtilsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/UtilsTest.java
index 20766b623b5fe..7c67c3da1606f 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/UtilsTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/UtilsTest.java
@@ -17,7 +17,8 @@
package org.apache.kafka.coordinator.group;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import com.dynatrace.hash4j.hashing.Hashing;
@@ -38,10 +39,10 @@ public class UtilsTest {
private static final String FOO_TOPIC_NAME = "foo";
private static final String BAR_TOPIC_NAME = "bar";
private static final int FOO_NUM_PARTITIONS = 2;
- private static final MetadataImage FOO_METADATA_IMAGE = new MetadataImageBuilder()
- .addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS)
- .addRacks()
- .build();
+ private static final CoordinatorMetadataImage FOO_METADATA_IMAGE = new MetadataImageBuilder()
+ .addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS)
+ .addRacks()
+ .buildCoordinatorMetadataImage();
@Test
void testNonExistingTopicName() {
@@ -170,7 +171,7 @@ void testComputeTopicHashWithDifferentRackOrder() {
@ParameterizedTest
@MethodSource("differentFieldGenerator")
- void testComputeTopicHashWithDifferentField(MetadataImage differentImage) {
+ void testComputeTopicHashWithDifferentField(CoordinatorMetadataImage differentImage) {
long result = Utils.computeTopicHash(FOO_TOPIC_NAME, FOO_METADATA_IMAGE);
assertNotEquals(
@@ -185,21 +186,21 @@ private static Stream differentFieldGenerator() {
new MetadataImageBuilder() // different topic id
.addTopic(Uuid.randomUuid(), FOO_TOPIC_NAME, FOO_NUM_PARTITIONS)
.addRacks()
- .build()
+ .buildCoordinatorMetadataImage()
),
Arguments.of(new MetadataImageBuilder() // different topic name
.addTopic(FOO_TOPIC_ID, "bar", FOO_NUM_PARTITIONS)
.addRacks()
- .build()
+ .buildCoordinatorMetadataImage()
),
Arguments.of(new MetadataImageBuilder() // different partitions
.addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, 1)
.addRacks()
- .build()
+ .buildCoordinatorMetadataImage()
),
Arguments.of(new MetadataImageBuilder() // different racks
.addTopic(FOO_TOPIC_ID, FOO_TOPIC_NAME, FOO_NUM_PARTITIONS)
- .build()
+ .buildCoordinatorMetadataImage()
)
);
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java
index 1038c3833d710..284bde18dfc24 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/OptimizedUniformAssignmentBuilderTest.java
@@ -17,7 +17,8 @@
package org.apache.kafka.coordinator.group.assignor;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorException;
@@ -66,7 +67,7 @@ public void testOneMemberNoTopicSubscription() {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = Map.of(
@@ -99,7 +100,7 @@ public void testOneMemberSubscribedToNonexistentTopic() {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = Map.of(
@@ -159,7 +160,7 @@ public void testFirstAssignmentTwoMembersTwoTopicsNoMemberRacks() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -218,7 +219,7 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -257,7 +258,7 @@ public void testValidityAndBalanceForLargeSampleSet() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -313,7 +314,7 @@ public void testReassignmentForTwoMembersTwoTopicsGivenUnbalancedPrevAssignment(
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -371,7 +372,7 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembersTwoTopics() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -438,7 +439,7 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -497,7 +498,7 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -553,7 +554,7 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -620,7 +621,7 @@ public void testReassignmentStickinessWhenAlreadyBalanced() {
HOMOGENEOUS,
invertedTargetAssignment(members)
);
- SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(metadataImage);
+ SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(new KRaftCoordinatorMetadataImage(metadataImage));
GroupAssignment computedAssignment = assignor.assign(
groupSpec,
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java
index f83655a3196c7..dcf2ebf593f27 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/RangeAssignorTest.java
@@ -17,7 +17,9 @@
package org.apache.kafka.coordinator.group.assignor;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -62,7 +64,7 @@ public class RangeAssignorTest {
@Test
public void testOneMemberNoTopic() {
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- MetadataImage.EMPTY
+ CoordinatorMetadataImage.EMPTY
);
Map members = Map.of(
@@ -100,7 +102,7 @@ public void testOneMemberSubscribedToNonExistentTopic() {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = Map.of(
@@ -152,7 +154,7 @@ public void testFirstAssignmentTwoMembersTwoTopicsSameSubscriptions() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -210,7 +212,7 @@ public void testFirstAssignmentThreeMembersThreeTopicsDifferentSubscriptions() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -270,7 +272,7 @@ public void testFirstAssignmentNumMembersGreaterThanNumPartitions() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -301,7 +303,7 @@ public void testStaticMembership() throws PartitionAssignorException {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = new TreeMap<>();
@@ -367,7 +369,7 @@ public void testMixedStaticMembership() throws PartitionAssignorException {
.addTopic(topic1Uuid, topic1Name, 5)
.build();
SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
// Initialize members with instance Ids.
@@ -484,7 +486,7 @@ public void testReassignmentNumMembersGreaterThanNumPartitionsWhenOneMemberAdded
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -543,7 +545,7 @@ public void testReassignmentWhenOnePartitionAddedForTwoMembersTwoTopics() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -607,7 +609,7 @@ public void testReassignmentWhenOneMemberAddedAfterInitialAssignmentWithTwoMembe
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -676,7 +678,7 @@ public void testReassignmentWhenOneMemberAddedAndOnePartitionAfterInitialAssignm
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -727,7 +729,7 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithTwoMem
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -791,7 +793,7 @@ public void testReassignmentWhenMultipleSubscriptionsRemovedAfterInitialAssignme
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java
index 9983eb3275379..bd54393fe367e 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/SimpleAssignorTest.java
@@ -17,7 +17,9 @@
package org.apache.kafka.coordinator.group.assignor;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -70,7 +72,7 @@ public void testName() {
@Test
public void testAssignWithEmptyMembers() {
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- MetadataImage.EMPTY
+ CoordinatorMetadataImage.EMPTY
);
GroupSpec groupSpec = new GroupSpecImpl(
@@ -104,7 +106,7 @@ public void testAssignWithNoSubscribedTopic() {
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = Map.of(
@@ -137,7 +139,7 @@ public void testAssignWithSubscribedToNonExistentTopic() {
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = Map.of(
@@ -193,7 +195,7 @@ public void testAssignWithTwoMembersAndTwoTopicsHomogeneous() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -243,7 +245,7 @@ public void testAssignWithTwoMembersAndTwoTopicsHomogeneousWithAllowedMap() {
)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -291,7 +293,7 @@ public void testAssignWithTwoMembersAndTwoTopicsHomogeneousWithNonAssignableTopi
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -345,7 +347,7 @@ public void testAssignWithThreeMembersThreeTopicsHeterogeneous() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -408,7 +410,7 @@ public void testAssignWithThreeMembersThreeTopicsHeterogeneousWithAllowedMap() {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -470,7 +472,7 @@ public void testAssignWithThreeMembersThreeTopicsHeterogeneousWithNonAssignableT
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -524,7 +526,7 @@ public void testAssignWithOneMemberNoAssignedTopicHeterogeneous() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -553,7 +555,7 @@ public void testIncrementalAssignmentIncreasingMembersHomogeneous() {
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Set topicsSubscription = new LinkedHashSet<>();
@@ -600,7 +602,7 @@ public void testIncrementalAssignmentDecreasingMembersHomogeneous() {
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Set topicsSubscription = new LinkedHashSet<>();
@@ -705,7 +707,7 @@ public void testAssignWithCurrentAssignmentHeterogeneous() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata1 = new SubscribedTopicDescriberImpl(
- metadataImage1
+ new KRaftCoordinatorMetadataImage(metadataImage1)
);
GroupAssignment computedAssignment1 = assignor.assign(
@@ -759,7 +761,7 @@ public void testAssignWithCurrentAssignmentHeterogeneous() {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata2 = new SubscribedTopicDescriberImpl(
- metadataImage2
+ new KRaftCoordinatorMetadataImage(metadataImage2)
);
GroupAssignment computedAssignment2 = assignor.assign(
@@ -775,11 +777,11 @@ public void testIncrementalAssignmentIncreasingMembersHeterogeneous() {
final int numPartitions = 24;
final int numMembers = 101;
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, numPartitions / 2)
.addTopic(TOPIC_2_UUID, TOPIC_2_NAME, numPartitions / 3)
.addTopic(TOPIC_3_UUID, TOPIC_3_NAME, numPartitions / 6)
- .build();
+ .buildCoordinatorMetadataImage();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
@@ -843,11 +845,11 @@ public void testIncrementalAssignmentDecreasingMembersHeterogeneous() {
final int numPartitions = 24;
final int numMembers = 101;
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, numPartitions / 2)
.addTopic(TOPIC_2_UUID, TOPIC_2_NAME, numPartitions / 3)
.addTopic(TOPIC_3_UUID, TOPIC_3_NAME, numPartitions / 6)
- .build();
+ .buildCoordinatorMetadataImage();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java
index 2c12ed467f17c..4c0059191870e 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/assignor/UniformHeterogeneousAssignmentBuilderTest.java
@@ -17,7 +17,8 @@
package org.apache.kafka.coordinator.group.assignor;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignorException;
@@ -94,7 +95,7 @@ public void testTwoMembersNoTopicSubscription() {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = new TreeMap<>();
@@ -131,7 +132,7 @@ public void testTwoMembersSubscribedToNonexistentTopics() {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
Map members = new TreeMap<>();
@@ -188,7 +189,7 @@ public void testFirstAssignmentTwoMembersTwoTopics() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -244,7 +245,7 @@ public void testFirstAssignmentNumMembersGreaterThanTotalNumPartitions() {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -313,7 +314,7 @@ public void testReassignmentForTwoMembersThreeTopicsGivenUnbalancedPrevAssignmen
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -377,7 +378,7 @@ public void testReassignmentWhenPartitionsAreAddedForTwoMembers() {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -442,7 +443,7 @@ public void testReassignmentWhenOneMemberAddedAndPartitionsAddedTwoMembersTwoTop
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -502,7 +503,7 @@ public void testReassignmentWhenOneMemberRemovedAfterInitialAssignmentWithThreeM
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -558,7 +559,7 @@ public void testReassignmentWhenOneSubscriptionRemovedAfterInitialAssignmentWith
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -617,7 +618,7 @@ public void testReassignmentWhenTopicPartitionsRunOutAndMembersHaveNoPartitions(
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@@ -666,7 +667,7 @@ public void testFirstAssignmentWithTwoMembersIncludingOneWithoutSubscriptions()
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
index 1ba83e10cd63e..187161b32fc44 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/classic/ClassicGroupTest.java
@@ -39,7 +39,8 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.OffsetAndMetadata;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
@@ -1461,7 +1462,7 @@ public void testFromConsumerGroupWithJoiningMember() {
newMember2,
logContext,
time,
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
ClassicGroup expectedClassicGroup = new ClassicGroup(
@@ -1592,7 +1593,7 @@ public void testFromConsumerGroupWithoutJoiningMember() {
null,
logContext,
time,
- metadataImage
+ new KRaftCoordinatorMetadataImage(metadataImage)
);
ClassicGroup expectedClassicGroup = new ClassicGroup(
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicMetadataTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicMetadataTest.java
index b8868a30a7f2f..78969773b8eaf 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicMetadataTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/SubscribedTopicMetadataTest.java
@@ -17,8 +17,8 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
-import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -32,7 +32,7 @@
public class SubscribedTopicMetadataTest {
private SubscribedTopicDescriberImpl subscribedTopicMetadata;
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
private final int numPartitions = 5;
@BeforeEach
@@ -43,7 +43,7 @@ public void setUp() {
String topicName = "topic" + i;
metadataImageBuilder.addTopic(topicId, topicName, numPartitions);
}
- metadataImage = metadataImageBuilder.addRacks().build();
+ metadataImage = metadataImageBuilder.addRacks().buildCoordinatorMetadataImage();
subscribedTopicMetadata = new SubscribedTopicDescriberImpl(metadataImage);
}
@@ -61,7 +61,7 @@ public void testNumberOfPartitions() {
assertEquals(-1, subscribedTopicMetadata.numPartitions(topicId));
// Test that the correct number of partitions are returned for a given topic ID.
- metadataImage.topics().topicsById().forEach((id, name) ->
+ metadataImage.topicIds().forEach(id ->
// Test that the correct number of partitions are returned for a given topic ID.
assertEquals(numPartitions, subscribedTopicMetadata.numPartitions(id))
);
@@ -73,7 +73,7 @@ public void testRacksForPartition() {
// Test empty set is returned when the topic ID doesn't exist.
assertEquals(Set.of(), subscribedTopicMetadata.racksForPartition(topicId, 0));
- metadataImage.topics().topicsById().forEach((id, name) -> {
+ metadataImage.topicIds().forEach(id -> {
// Test empty set is returned when the partition ID doesn't exist.
assertEquals(Set.of(), subscribedTopicMetadata.racksForPartition(id, 10));
@@ -87,10 +87,10 @@ public void testEquals() {
assertEquals(new SubscribedTopicDescriberImpl(metadataImage), subscribedTopicMetadata);
Uuid topicId = Uuid.randomUuid();
- MetadataImage metadataImage2 = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage2 = new MetadataImageBuilder()
.addTopic(topicId, "newTopic", 5)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
assertNotEquals(new SubscribedTopicDescriberImpl(metadataImage2), subscribedTopicMetadata);
}
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java
index 59541ebad6a23..3886e634bfd2f 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TargetAssignmentBuilderTest.java
@@ -17,15 +17,16 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.AssignmentTestUtil;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
import org.apache.kafka.coordinator.group.api.assignor.PartitionAssignor;
import org.apache.kafka.coordinator.group.api.assignor.SubscriptionType;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
import org.apache.kafka.coordinator.group.modern.consumer.ResolvedRegularExpression;
-import org.apache.kafka.image.MetadataImage;
import org.junit.jupiter.api.Test;
@@ -212,8 +213,8 @@ private MemberSubscriptionAndAssignmentImpl newMemberSubscriptionAndAssignment(
}
public TargetAssignmentBuilder.TargetAssignmentResult build() {
- MetadataImage metadataImage = metadataImageBuilder.build();
- TopicIds.TopicResolver topicResolver = new TopicIds.CachedTopicResolver(metadataImage.topics());
+ CoordinatorMetadataImage cooridnatorMetadataImage = new KRaftCoordinatorMetadataImage(metadataImageBuilder.build());
+ TopicIds.TopicResolver topicResolver = new TopicIds.CachedTopicResolver(cooridnatorMetadataImage);
// Prepare expected member specs.
Map memberSubscriptions = new HashMap<>();
@@ -251,7 +252,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() {
});
// Prepare the expected subscription topic metadata.
- SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(metadataImage);
+ SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(cooridnatorMetadataImage);
SubscriptionType subscriptionType = HOMOGENEOUS;
// Prepare the member assignments per topic partition.
@@ -278,7 +279,7 @@ public TargetAssignmentBuilder.TargetAssignmentResult build() {
.withSubscriptionType(subscriptionType)
.withTargetAssignment(targetAssignment)
.withInvertedTargetAssignment(invertedTargetAssignment)
- .withMetadataImage(metadataImage)
+ .withMetadataImage(cooridnatorMetadataImage)
.withResolvedRegularExpressions(resolvedRegularExpressions);
// Add the updated members or delete the deleted members.
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java
index 653233923d0db..c44efa7b5dbd6 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/TopicIdsTest.java
@@ -17,8 +17,9 @@
package org.apache.kafka.coordinator.group.modern;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
-import org.apache.kafka.image.TopicsImage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.junit.jupiter.api.Test;
@@ -34,12 +35,12 @@ public class TopicIdsTest {
@Test
public void testTopicNamesCannotBeNull() {
- assertThrows(NullPointerException.class, () -> new TopicIds(null, TopicsImage.EMPTY));
+ assertThrows(NullPointerException.class, () -> new TopicIds(null, CoordinatorMetadataImage.EMPTY));
}
@Test
public void testTopicsImageCannotBeNull() {
- assertThrows(NullPointerException.class, () -> new TopicIds(Set.of(), (TopicsImage) null));
+ assertThrows(NullPointerException.class, () -> new TopicIds(Set.of(), (CoordinatorMetadataImage) null));
}
@Test
@@ -50,14 +51,14 @@ public void testTopicResolverCannotBeNull() {
@Test
public void testSize() {
Set topicNames = Set.of("foo", "bar", "baz");
- Set topicIds = new TopicIds(topicNames, TopicsImage.EMPTY);
+ Set topicIds = new TopicIds(topicNames, CoordinatorMetadataImage.EMPTY);
assertEquals(topicNames.size(), topicIds.size());
}
@Test
public void testIsEmpty() {
Set topicNames = Set.of();
- Set topicIds = new TopicIds(topicNames, TopicsImage.EMPTY);
+ Set topicIds = new TopicIds(topicNames, CoordinatorMetadataImage.EMPTY);
assertEquals(topicNames.size(), topicIds.size());
}
@@ -67,14 +68,13 @@ public void testContains() {
Uuid barUuid = Uuid.randomUuid();
Uuid bazUuid = Uuid.randomUuid();
Uuid quxUuid = Uuid.randomUuid();
- TopicsImage topicsImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooUuid, "foo", 3)
.addTopic(barUuid, "bar", 3)
.addTopic(bazUuid, "qux", 3)
- .build()
- .topics();
+ .build());
- Set topicIds = new TopicIds(Set.of("foo", "bar", "baz"), topicsImage);
+ Set topicIds = new TopicIds(Set.of("foo", "bar", "baz"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@@ -88,15 +88,14 @@ public void testContainsAll() {
Uuid barUuid = Uuid.randomUuid();
Uuid bazUuid = Uuid.randomUuid();
Uuid quxUuid = Uuid.randomUuid();
- TopicsImage topicsImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooUuid, "foo", 3)
.addTopic(barUuid, "bar", 3)
.addTopic(bazUuid, "baz", 3)
.addTopic(quxUuid, "qux", 3)
- .build()
- .topics();
+ .build());
- Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
+ Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@@ -112,14 +111,13 @@ public void testContainsAllOneTopicConversionFails() {
Uuid barUuid = Uuid.randomUuid();
Uuid bazUuid = Uuid.randomUuid();
Uuid quxUuid = Uuid.randomUuid();
- TopicsImage topicsImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooUuid, "foo", 3)
.addTopic(barUuid, "bar", 3)
.addTopic(bazUuid, "baz", 3)
- .build()
- .topics();
+ .build());
- Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
+ Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@@ -134,15 +132,14 @@ public void testIterator() {
Uuid barUuid = Uuid.randomUuid();
Uuid bazUuid = Uuid.randomUuid();
Uuid quxUuid = Uuid.randomUuid();
- TopicsImage topicsImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooUuid, "foo", 3)
.addTopic(barUuid, "bar", 3)
.addTopic(bazUuid, "baz", 3)
.addTopic(quxUuid, "qux", 3)
- .build()
- .topics();
+ .build());
- Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
+ Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
Set expectedIds = Set.of(fooUuid, barUuid, bazUuid, quxUuid);
Set actualIds = new HashSet<>(topicIds);
@@ -157,15 +154,14 @@ public void testIteratorOneTopicConversionFails() {
Uuid barUuid = Uuid.randomUuid();
Uuid bazUuid = Uuid.randomUuid();
Uuid qux = Uuid.randomUuid();
- TopicsImage topicsImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(fooUuid, "foo", 3)
.addTopic(barUuid, "bar", 3)
.addTopic(bazUuid, "baz", 3)
.addTopic(qux, "qux", 3)
- .build()
- .topics();
+ .build());
- Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "quux"), topicsImage);
+ Set topicIds = new TopicIds(Set.of("foo", "bar", "baz", "quux"), metadataImage);
Set expectedIds = Set.of(fooUuid, barUuid, bazUuid);
Set actualIds = new HashSet<>(topicIds);
@@ -175,19 +171,12 @@ public void testIteratorOneTopicConversionFails() {
@Test
public void testEquals() {
Uuid topicId = Uuid.randomUuid();
- TopicIds topicIds1 = new TopicIds(Set.of("topic"),
- new MetadataImageBuilder()
- .addTopic(topicId, "topicId", 3)
- .build()
- .topics()
- );
-
- TopicIds topicIds2 = new TopicIds(Set.of("topic"),
- new MetadataImageBuilder()
- .addTopic(topicId, "topicId", 3)
- .build()
- .topics()
- );
+ KRaftCoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
+ .addTopic(topicId, "topicId", 3)
+ .build());
+
+ TopicIds topicIds1 = new TopicIds(Set.of("topic"), metadataImage);
+ TopicIds topicIds2 = new TopicIds(Set.of("topic"), metadataImage);
assertEquals(topicIds1, topicIds2);
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java
index 47b50699ab143..c30b46fd753ec 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupMemberTest.java
@@ -19,7 +19,8 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ConsumerGroupDescribeResponseData;
import org.apache.kafka.common.message.JoinGroupRequestData;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupCurrentMemberAssignmentValue;
import org.apache.kafka.coordinator.group.generated.ConsumerGroupMemberMetadataValue;
import org.apache.kafka.coordinator.group.modern.Assignment;
@@ -292,7 +293,7 @@ public void testAsConsumerGroupDescribeMember(boolean withClassicMemberMetadata)
.setSupportedProtocols(toClassicProtocolCollection("range")) : null)
.build();
- ConsumerGroupDescribeResponseData.Member actual = member.asConsumerGroupDescribeMember(targetAssignment, metadataImage.topics());
+ ConsumerGroupDescribeResponseData.Member actual = member.asConsumerGroupDescribeMember(targetAssignment, new KRaftCoordinatorMetadataImage(metadataImage));
ConsumerGroupDescribeResponseData.Member expected = new ConsumerGroupDescribeResponseData.Member()
.setMemberId(memberId)
.setMemberEpoch(epoch)
@@ -330,7 +331,7 @@ public void testAsConsumerGroupDescribeWithTargetAssignmentNull() {
.build();
ConsumerGroupDescribeResponseData.Member consumerGroupDescribeMember = member.asConsumerGroupDescribeMember(
- null, new MetadataImageBuilder().build().topics());
+ null, new KRaftCoordinatorMetadataImage(new MetadataImageBuilder().build()));
assertEquals(new ConsumerGroupDescribeResponseData.Assignment(), consumerGroupDescribeMember.targetAssignment());
}
@@ -351,9 +352,9 @@ public void testAsConsumerGroupDescribeWithTopicNameNotFound() {
.setSubscribedTopicRegex("")
.setMemberType((byte) 1);
ConsumerGroupDescribeResponseData.Member actual = member.asConsumerGroupDescribeMember(null,
- new MetadataImageBuilder()
+ new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "foo", 3)
- .build().topics()
+ .build())
);
assertEquals(expected, actual);
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
index 7850fde746d66..d75029c2799d0 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/consumer/ConsumerGroupTest.java
@@ -33,10 +33,12 @@
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.Group;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.OffsetAndMetadata;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
@@ -48,7 +50,6 @@
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.ModernGroup;
import org.apache.kafka.coordinator.group.modern.SubscriptionCount;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.Test;
@@ -1062,7 +1063,7 @@ public void testAsDescribedGroup() {
.setMemberType((byte) 1)
));
ConsumerGroupDescribeResponseData.DescribedGroup actual = group.asDescribedGroup(1, "",
- new MetadataImageBuilder().build().topics());
+ new KRaftCoordinatorMetadataImage(new MetadataImageBuilder().build()));
assertEquals(expected, actual);
}
@@ -1261,11 +1262,11 @@ public void testFromClassicGroup() {
Uuid barTopicId = Uuid.randomUuid();
String barTopicName = "bar";
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(fooTopicId, fooTopicName, 1)
.addTopic(barTopicId, barTopicName, 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
ClassicGroup classicGroup = new ClassicGroup(
logContext,
@@ -2040,11 +2041,11 @@ public void testComputeSubscribedRegularExpressions() {
@Test
public void testComputeMetadataHash() {
- MetadataImage metadataImage = new MetadataImageBuilder()
+ CoordinatorMetadataImage metadataImage = new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "foo", 1)
.addTopic(Uuid.randomUuid(), "bar", 1)
.addRacks()
- .build();
+ .buildCoordinatorMetadataImage();
Map cache = new HashMap<>();
assertEquals(
computeGroupHash(Map.of(
@@ -2084,11 +2085,11 @@ public void testComputeMetadataHashUseCacheData() {
"bar", new SubscriptionCount(1, 0)
),
cache,
- new MetadataImageBuilder()
+ new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "foo", 1)
.addTopic(Uuid.randomUuid(), "bar", 1)
.addRacks()
- .build()
+ .build())
)
);
assertEquals(
@@ -2120,11 +2121,11 @@ public void testComputeMetadataHashIgnoreTopicHashIfItIsNotInMetadataImage() {
"bar", new SubscriptionCount(1, 0)
),
cache,
- new MetadataImageBuilder()
+ new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "foo", 1)
.addTopic(Uuid.randomUuid(), "bar", 1)
.addRacks()
- .build()
+ .build())
)
);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java
index 2975d4793942f..67cfb75661272 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupBuilder.java
@@ -20,7 +20,6 @@
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.group.GroupCoordinatorRecordHelpers;
import org.apache.kafka.coordinator.group.modern.Assignment;
-import org.apache.kafka.image.TopicsImage;
import java.util.ArrayList;
import java.util.HashMap;
@@ -62,7 +61,7 @@ public ShareGroupBuilder withAssignmentEpoch(int assignmentEpoch) {
return this;
}
- public List build(TopicsImage topicsImage) {
+ public List build() {
List records = new ArrayList<>();
// Add subscription records for members.
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java
index 150269447218f..54912395430ed 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupMemberTest.java
@@ -18,7 +18,8 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.ShareGroupDescribeResponseData;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.generated.ShareGroupMemberMetadataValue;
import org.apache.kafka.image.MetadataImage;
@@ -169,7 +170,7 @@ public void testAsShareGroupDescribeMember() {
mkTopicAssignment(topicId1, 0, 1, 2)))
.build();
- ShareGroupDescribeResponseData.Member actual = member.asShareGroupDescribeMember(metadataImage.topics());
+ ShareGroupDescribeResponseData.Member actual = member.asShareGroupDescribeMember(new KRaftCoordinatorMetadataImage(metadataImage));
ShareGroupDescribeResponseData.Member expected = new ShareGroupDescribeResponseData.Member()
.setMemberId(memberId)
.setMemberEpoch(epoch)
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java
index 5c4788b997197..aeaebcb2d194b 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/modern/share/ShareGroupTest.java
@@ -25,8 +25,9 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.Group;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.modern.Assignment;
import org.apache.kafka.coordinator.group.modern.MemberState;
import org.apache.kafka.coordinator.group.modern.share.ShareGroup.ShareGroupState;
@@ -462,7 +463,7 @@ public void testAsDescribedGroup() {
new ShareGroupDescribeResponseData.Member().setMemberId("member2")
));
ShareGroupDescribeResponseData.DescribedGroup actual = shareGroup.asDescribedGroup(1, "assignorName",
- new MetadataImageBuilder().build().topics());
+ new KRaftCoordinatorMetadataImage(new MetadataImageBuilder().build()));
assertEquals(expected, actual);
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java
index 8966c936356d3..a76e282be6ec5 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/StreamsGroupTest.java
@@ -30,7 +30,8 @@
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.annotation.ApiKeyVersionsSource;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.OffsetAndMetadata;
import org.apache.kafka.coordinator.group.OffsetExpirationCondition;
import org.apache.kafka.coordinator.group.OffsetExpirationConditionImpl;
@@ -921,7 +922,7 @@ public void testComputeMetadataHash() {
StreamsTopology topology = mock(StreamsTopology.class);
when(topology.requiredTopics()).thenReturn(Set.of("topic1"));
- long metadataHash = streamsGroup.computeMetadataHash(metadataImage, new HashMap<>(), topology);
+ long metadataHash = streamsGroup.computeMetadataHash(new KRaftCoordinatorMetadataImage(metadataImage), new HashMap<>(), topology);
// The metadata hash means no topic.
assertNotEquals(0, metadataHash);
}
@@ -988,7 +989,7 @@ public void testIsSubscribedToTopic() {
.addTopic(Uuid.randomUuid(), "test-topic2", 1)
.build();
- streamsGroup.setConfiguredTopology(InternalTopicManager.configureTopics(logContext, 0, topology, metadataImage.topics()));
+ streamsGroup.setConfiguredTopology(InternalTopicManager.configureTopics(logContext, 0, topology, new KRaftCoordinatorMetadataImage(metadataImage)));
assertTrue(streamsGroup.isSubscribedToTopic("test-topic1"));
assertTrue(streamsGroup.isSubscribedToTopic("test-topic2"));
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java
index ad69879af3214..7d8ba55e24eb8 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TargetAssignmentBuilderTest.java
@@ -17,8 +17,10 @@
package org.apache.kafka.coordinator.group.streams;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.generated.StreamsGroupMemberMetadataValue;
import org.apache.kafka.coordinator.group.streams.TaskAssignmentTestUtil.TaskRole;
import org.apache.kafka.coordinator.group.streams.assignor.AssignmentMemberSpec;
@@ -28,7 +30,6 @@
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredTopology;
-import org.apache.kafka.image.MetadataImage;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
@@ -801,7 +802,7 @@ public org.apache.kafka.coordinator.group.streams.TargetAssignmentBuilder.Target
}
});
- MetadataImage metadataImage = topicsImageBuilder.build();
+ CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(topicsImageBuilder.build());
// Prepare the expected topology metadata.
TopologyMetadata topologyMetadata = new TopologyMetadata(metadataImage, subtopologies);
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TopologyMetadataTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TopologyMetadataTest.java
index fa047be2a8a02..57aa505765721 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TopologyMetadataTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/TopologyMetadataTest.java
@@ -17,10 +17,11 @@
package org.apache.kafka.coordinator.group.streams;
import org.apache.kafka.common.Uuid;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredInternalTopic;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
-import org.apache.kafka.image.MetadataImage;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
@@ -41,16 +42,16 @@
class TopologyMetadataTest {
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
private SortedMap subtopologyMap;
private TopologyMetadata topologyMetadata;
@BeforeEach
void setUp() {
- metadataImage = new MetadataImageBuilder()
+ metadataImage = new KRaftCoordinatorMetadataImage(new MetadataImageBuilder()
.addTopic(Uuid.randomUuid(), "source_topic", 3)
.addTopic(Uuid.randomUuid(), "repartition_source_topic", 4)
- .build();
+ .build());
subtopologyMap = new TreeMap<>();
topologyMetadata = new TopologyMetadata(metadataImage, subtopologyMap);
}
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java
index eea0d0051f388..a59bab697f844 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/EndpointToPartitionsManagerTest.java
@@ -19,7 +19,8 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.message.StreamsGroupHeartbeatResponseData;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.streams.StreamsGroup;
import org.apache.kafka.coordinator.group.streams.StreamsGroupMember;
import org.apache.kafka.coordinator.group.streams.TasksTuple;
@@ -97,7 +98,7 @@ void testEndpointToPartitionsWithStandbyTaskAssignments() {
when(configuredTopology.subtopologies()).thenReturn(Optional.of(configuredSubtopologyMap));
StreamsGroupHeartbeatResponseData.EndpointToPartitions result =
- EndpointToPartitionsManager.endpointToPartitions(streamsGroupMember, responseEndpoint, streamsGroup, metadataImage);
+ EndpointToPartitionsManager.endpointToPartitions(streamsGroupMember, responseEndpoint, streamsGroup, new KRaftCoordinatorMetadataImage(metadataImage));
assertEquals(responseEndpoint, result.userEndpoint());
assertEquals(1, result.activePartitions().size());
@@ -137,7 +138,7 @@ void testEndpointToPartitionsWithTwoTopicsAndDifferentPartitions(int topicAParti
configuredSubtopologyOneMap.put("0", configuredSubtopologyOne);
when(configuredTopology.subtopologies()).thenReturn(Optional.of(configuredSubtopologyOneMap));
- StreamsGroupHeartbeatResponseData.EndpointToPartitions result = EndpointToPartitionsManager.endpointToPartitions(streamsGroupMember, responseEndpoint, streamsGroup, metadataImage);
+ StreamsGroupHeartbeatResponseData.EndpointToPartitions result = EndpointToPartitionsManager.endpointToPartitions(streamsGroupMember, responseEndpoint, streamsGroup, new KRaftCoordinatorMetadataImage(metadataImage));
assertEquals(responseEndpoint, result.userEndpoint());
assertEquals(2, result.activePartitions().size());
diff --git a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java
index 07402ac8ebf0c..8c7c389e562aa 100644
--- a/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java
+++ b/group-coordinator/src/test/java/org/apache/kafka/coordinator/group/streams/topics/InternalTopicManagerTest.java
@@ -22,7 +22,8 @@
import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicConfigCollection;
import org.apache.kafka.common.requests.StreamsGroupHeartbeatResponse.Status;
import org.apache.kafka.common.utils.LogContext;
-import org.apache.kafka.coordinator.group.MetadataImageBuilder;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.MetadataImageBuilder;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue;
import org.apache.kafka.coordinator.group.generated.StreamsGroupTopologyValue.Subtopology;
import org.apache.kafka.coordinator.group.streams.StreamsTopology;
@@ -60,7 +61,7 @@ void testConfigureTopicsSetsConfigurationExceptionWhenSourceTopicIsMissing() {
// SOURCE_TOPIC_2 is missing from topicMetadata
StreamsTopology topology = makeTestTopology();
- final ConfiguredTopology configuredTopology = InternalTopicManager.configureTopics(new LogContext(), 0, topology, metadataImage.topics());
+ final ConfiguredTopology configuredTopology = InternalTopicManager.configureTopics(new LogContext(), 0, topology, new KRaftCoordinatorMetadataImage(metadataImage));
assertEquals(Optional.empty(), configuredTopology.subtopologies());
assertTrue(configuredTopology.topicConfigurationException().isPresent());
@@ -77,7 +78,7 @@ void testConfigureTopics() {
.build();
StreamsTopology topology = makeTestTopology();
- ConfiguredTopology configuredTopology = InternalTopicManager.configureTopics(new LogContext(), 0, topology, metadataImage.topics());
+ ConfiguredTopology configuredTopology = InternalTopicManager.configureTopics(new LogContext(), 0, topology, new KRaftCoordinatorMetadataImage(metadataImage));
final Map internalTopicsToBeCreated = configuredTopology.internalTopicsToBeCreated();
assertEquals(2, internalTopicsToBeCreated.size());
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java
index b1efff5cf95f9..555c92457f831 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/AssignorBenchmarkUtils.java
@@ -19,6 +19,8 @@
import org.apache.kafka.common.Uuid;
import org.apache.kafka.common.metadata.PartitionRecord;
import org.apache.kafka.common.metadata.TopicRecord;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -94,7 +96,7 @@ public static List createTopicNames(int topicCount) {
* @return A TopicsImage containing the topic ids, names and partition counts from the
* subscription metadata.
*/
- public static MetadataImage createMetadataImage(
+ public static CoordinatorMetadataImage createMetadataImage(
List allTopicNames,
int partitionsPerTopic
) {
@@ -109,7 +111,7 @@ public static MetadataImage createMetadataImage(
);
}
- return delta.apply(MetadataProvenance.EMPTY);
+ return new KRaftCoordinatorMetadataImage(delta.apply(MetadataProvenance.EMPTY));
}
/**
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java
index 85d42709788d2..b96d718c654d6 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ServerSideAssignorBenchmark.java
@@ -17,6 +17,7 @@
package org.apache.kafka.jmh.assignor;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -32,7 +33,6 @@
import org.apache.kafka.coordinator.group.modern.SubscribedTopicDescriberImpl;
import org.apache.kafka.coordinator.group.modern.TopicIds;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
-import org.apache.kafka.image.MetadataImage;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -122,7 +122,7 @@ public enum AssignmentType {
private List allTopicNames = List.of();
- private MetadataImage metadataImage = MetadataImage.EMPTY;
+ private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
private TopicIds.TopicResolver topicResolver;
@@ -148,7 +148,7 @@ private void setupTopics() {
int partitionsPerTopic = (memberCount * partitionsToMemberRatio) / topicCount;
metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionsPerTopic);
- topicResolver = new TopicIds.CachedTopicResolver(metadataImage.topics());
+ topicResolver = new TopicIds.CachedTopicResolver(metadataImage);
subscribedTopicDescriber = new SubscribedTopicDescriberImpl(metadataImage);
}
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ShareGroupAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ShareGroupAssignorBenchmark.java
index 99661a9e9089c..84a59560e0a97 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ShareGroupAssignorBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/ShareGroupAssignorBenchmark.java
@@ -17,6 +17,7 @@
package org.apache.kafka.jmh.assignor;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -31,7 +32,6 @@
import org.apache.kafka.coordinator.group.modern.SubscribedTopicDescriberImpl;
import org.apache.kafka.coordinator.group.modern.TopicIds;
import org.apache.kafka.coordinator.group.modern.share.ShareGroupMember;
-import org.apache.kafka.image.MetadataImage;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -136,8 +136,8 @@ public void setup() {
private void setupTopics() {
allTopicNames = AssignorBenchmarkUtils.createTopicNames(topicCount);
- MetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
- topicResolver = new TopicIds.CachedTopicResolver(metadataImage.topics());
+ CoordinatorMetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
+ topicResolver = new TopicIds.CachedTopicResolver(metadataImage);
subscribedTopicDescriber = new SubscribedTopicDescriberImpl(metadataImage);
}
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java
index 605c6b89cafb3..3293c0223f2a5 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/StreamsStickyAssignorBenchmark.java
@@ -16,6 +16,7 @@
*/
package org.apache.kafka.jmh.assignor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.streams.StreamsGroupMember;
import org.apache.kafka.coordinator.group.streams.TopologyMetadata;
import org.apache.kafka.coordinator.group.streams.assignor.AssignmentMemberSpec;
@@ -27,7 +28,6 @@
import org.apache.kafka.coordinator.group.streams.assignor.TaskAssignor;
import org.apache.kafka.coordinator.group.streams.assignor.TopologyDescriber;
import org.apache.kafka.coordinator.group.streams.topics.ConfiguredSubtopology;
-import org.apache.kafka.image.MetadataImage;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -100,7 +100,7 @@ public void setup() {
SortedMap subtopologyMap = StreamsAssignorBenchmarkUtils.createSubtopologyMap(partitionCount, allTopicNames);
- MetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
+ CoordinatorMetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
topologyDescriber = new TopologyMetadata(metadataImage, subtopologyMap);
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java
index 7e21636cc4652..c669737de36d2 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/assignor/TargetAssignmentBuilderBenchmark.java
@@ -17,6 +17,7 @@
package org.apache.kafka.jmh.assignor;
import org.apache.kafka.common.Uuid;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.api.assignor.GroupAssignment;
import org.apache.kafka.coordinator.group.api.assignor.GroupSpec;
import org.apache.kafka.coordinator.group.api.assignor.MemberAssignment;
@@ -29,7 +30,6 @@
import org.apache.kafka.coordinator.group.modern.TargetAssignmentBuilder;
import org.apache.kafka.coordinator.group.modern.TopicIds;
import org.apache.kafka.coordinator.group.modern.consumer.ConsumerGroupMember;
-import org.apache.kafka.image.MetadataImage;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -91,7 +91,7 @@ public class TargetAssignmentBuilderBenchmark {
private List allTopicNames = List.of();
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
private TopicIds.TopicResolver topicResolver;
@@ -127,7 +127,7 @@ private void setupTopics() {
int partitionsPerTopic = (memberCount * partitionsToMemberRatio) / topicCount;
metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionsPerTopic);
- topicResolver = new TopicIds.CachedTopicResolver(metadataImage.topics());
+ topicResolver = new TopicIds.CachedTopicResolver(metadataImage);
subscribedTopicDescriber = new SubscribedTopicDescriberImpl(metadataImage);
}
diff --git a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/RegexResolutionBenchmark.java b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/RegexResolutionBenchmark.java
index 35ee42836c820..aa91e88e9a5ef 100644
--- a/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/RegexResolutionBenchmark.java
+++ b/jmh-benchmarks/src/main/java/org/apache/kafka/jmh/coordinator/RegexResolutionBenchmark.java
@@ -20,6 +20,7 @@
import org.apache.kafka.common.metadata.TopicRecord;
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
import org.apache.kafka.coordinator.group.GroupMetadataManager;
import org.apache.kafka.image.MetadataDelta;
import org.apache.kafka.image.MetadataImage;
@@ -123,7 +124,7 @@ public void run() {
GROUP_ID,
LOG,
TIME,
- image,
+ new KRaftCoordinatorMetadataImage(image),
Optional.empty(),
regexes
);
diff --git a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java
index 3da349ce0f831..9154e799a3367 100644
--- a/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java
+++ b/metadata/src/test/java/org/apache/kafka/image/TopicsImageTest.java
@@ -893,5 +893,8 @@ public void testTopicIdToNameView() {
assertEquals(expectedNames, names);
assertThrows(UnsupportedOperationException.class, () -> map.remove(FOO_UUID));
assertThrows(UnsupportedOperationException.class, () -> map.put(FOO_UUID, "bar"));
+
+ var result = IMAGE1.topicIdToNameView().get("zar");
+ assertNull(result);
}
}
diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java
index dda063ab4df85..a5c29ff6807be 100644
--- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java
+++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinator.java
@@ -30,8 +30,9 @@
import org.apache.kafka.common.message.WriteShareGroupStateResponseData;
import org.apache.kafka.common.requests.RequestContext;
import org.apache.kafka.common.utils.BufferSupplier;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
+import org.apache.kafka.image.FeaturesImage;
import org.apache.kafka.server.share.SharePartitionKey;
import java.util.OptionalInt;
@@ -135,11 +136,12 @@ public interface ShareCoordinator {
/**
* A new metadata image is available.
*
- * @param newImage The new metadata image.
- * @param delta The metadata delta.
+ * @param newImage The new metadata image.
+ * @param newFeaturesImage The features image.
+ * @param delta The metadata delta.
*/
void onNewMetadataImage(
- MetadataImage newImage,
- MetadataDelta delta
+ CoordinatorMetadataImage newImage,
+ FeaturesImage newFeaturesImage, CoordinatorMetadataDelta delta
);
}
diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java
index 44315418c66f6..3a85cc3f1feda 100644
--- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java
+++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorService.java
@@ -45,6 +45,8 @@
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.coordinator.common.runtime.CoordinatorEventProcessor;
import org.apache.kafka.coordinator.common.runtime.CoordinatorLoader;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntimeMetrics;
@@ -52,8 +54,7 @@
import org.apache.kafka.coordinator.common.runtime.MultiThreadedEventProcessor;
import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.FeaturesImage;
import org.apache.kafka.server.common.ShareVersion;
import org.apache.kafka.server.record.BrokerCompressionType;
import org.apache.kafka.server.share.SharePartitionKey;
@@ -1096,10 +1097,10 @@ public void onTopicsDeleted(Set deletedTopicIds, BufferSupplier bufferSupp
}
@Override
- public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
+ public void onNewMetadataImage(CoordinatorMetadataImage newImage, FeaturesImage newFeaturesImage, CoordinatorMetadataDelta delta) {
throwIfNotActive();
this.runtime.onNewMetadataImage(newImage, delta);
- boolean enabled = isShareGroupsEnabled(newImage);
+ boolean enabled = isShareGroupsEnabled(newFeaturesImage);
// enabled shouldRunJob result (XOR)
// 0 0 no op on flag, do not call jobs
// 0 1 disable flag, do not call jobs => action
@@ -1127,9 +1128,9 @@ private void throwIfNotActive() {
}
}
- private boolean isShareGroupsEnabled(MetadataImage image) {
+ private boolean isShareGroupsEnabled(FeaturesImage image) {
return shareGroupConfigEnabledSupplier.get() || ShareVersion.fromFeatureLevel(
- image.features().finalizedVersions().getOrDefault(ShareVersion.FEATURE_NAME, (short) 0)
+ image.finalizedVersions().getOrDefault(ShareVersion.FEATURE_NAME, (short) 0)
).supportsShareGroups();
}
diff --git a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java
index ba47582fe1666..5dad56f48bce1 100644
--- a/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java
+++ b/share-coordinator/src/main/java/org/apache/kafka/coordinator/share/ShareCoordinatorShard.java
@@ -41,6 +41,8 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.coordinator.common.runtime.CoordinatorExecutor;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
@@ -55,8 +57,6 @@
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetricsShard;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.share.persister.PartitionFactory;
@@ -83,7 +83,7 @@ public class ShareCoordinatorShard implements CoordinatorShard leaderEpochMap;
private final TimelineHashMap snapshotUpdateCount;
private final TimelineHashMap stateEpochMap;
- private MetadataImage metadataImage;
+ private CoordinatorMetadataImage metadataImage;
private final ShareCoordinatorOffsetsManager offsetsManager;
private final Time time;
@@ -206,13 +206,13 @@ public ShareCoordinatorShard build() {
}
@Override
- public void onLoaded(MetadataImage newImage) {
+ public void onLoaded(CoordinatorMetadataImage newImage) {
this.metadataImage = newImage;
coordinatorMetrics.activateMetricsShard(metricsShard);
}
@Override
- public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
+ public void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {
this.metadataImage = newImage;
}
@@ -769,8 +769,9 @@ private Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty() ||
+ topicMetadataOp.get().partitionCount() <= partitionId) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(getWriteErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
@@ -816,8 +817,9 @@ private Optional maybeGetReadStateError(ReadSha
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
- if (metadataImage.topics().getTopic(topicId) == null ||
- metadataImage.topics().getPartition(topicId, partitionId) == null) {
+ Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty() ||
+ topicMetadataOp.get().partitionCount() <= partitionId) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
@@ -849,8 +851,9 @@ private Optional maybeGetReadStateSummar
return Optional.of(ReadShareGroupStateSummaryResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
- if (metadataImage.topics().getTopic(topicId) == null ||
- metadataImage.topics().getPartition(topicId, partitionId) == null) {
+ Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty() ||
+ topicMetadataOp.get().partitionCount() <= partitionId) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(ReadShareGroupStateSummaryResponse.toErrorResponseData(topicId, partitionId, Errors.UNKNOWN_TOPIC_OR_PARTITION, Errors.UNKNOWN_TOPIC_OR_PARTITION.message()));
}
@@ -880,8 +883,9 @@ private Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty() ||
+ topicMetadataOp.get().partitionCount() <= partitionId) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(getDeleteErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
@@ -917,8 +921,9 @@ private Optional topicMetadataOp = metadataImage.topicMetadata(topicId);
+ if (topicMetadataOp.isEmpty() ||
+ topicMetadataOp.get().partitionCount() <= partitionId) {
log.error("Topic/TopicPartition not found in metadata image.");
return Optional.of(getInitializeErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java
index f7c6e32ceb390..d44735f048fa9 100644
--- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java
+++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorServiceTest.java
@@ -41,12 +41,13 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataDelta;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRuntime;
import org.apache.kafka.coordinator.common.runtime.PartitionWriter;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
-import org.apache.kafka.image.MetadataDelta;
-import org.apache.kafka.image.MetadataImage;
+import org.apache.kafka.image.FeaturesImage;
import org.apache.kafka.server.common.ShareVersion;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.util.FutureUtils;
@@ -1485,7 +1486,7 @@ public void testRecordPruningTaskPeriodicityWithAllSuccess() throws Exception {
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1580,7 +1581,7 @@ public void testRecordPruningTaskPeriodicityWithSomeFailures() throws Exception
));
service.startup(() -> 2);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1642,7 +1643,7 @@ public void testRecordPruningTaskException() throws Exception {
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1695,7 +1696,7 @@ public void testRecordPruningTaskSuccess() throws Exception {
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1746,7 +1747,7 @@ public void testRecordPruningTaskEmptyOffsetReturned() throws Exception {
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
@@ -1810,7 +1811,7 @@ public void testRecordPruningTaskRepeatedSameOffsetForTopic() throws Exception {
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1886,7 +1887,7 @@ public void testRecordPruningTaskRetriesRepeatedSameOffsetForTopic() throws Exce
));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("write-state-record-prune"),
@@ -1946,7 +1947,7 @@ public void testColdPartitionSnapshotTaskPeriodicityWithAllSuccess() throws Exce
)).thenReturn(List.of(CompletableFuture.completedFuture(null)));
service.startup(() -> 1);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteOperation(
eq("snapshot-cold-partitions"),
@@ -2004,7 +2005,7 @@ public void testColdPartitionSnapshotTaskPeriodicityWithSomeFailures() throws Ex
));
service.startup(() -> 2);
- service.onNewMetadataImage(mock(MetadataImage.class), mock(MetadataDelta.class));
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mock(FeaturesImage.class), mock(CoordinatorMetadataDelta.class));
verify(runtime, times(0))
.scheduleWriteAllOperation(
eq("snapshot-cold-partitions"),
@@ -2069,11 +2070,11 @@ public void testPeriodicJobsDoNotRunWhenShareGroupsDisabled() throws Interrupted
service.startup(() -> 1);
- MetadataImage mockedImage = mock(MetadataImage.class, RETURNS_DEEP_STUBS);
+ FeaturesImage mockedFeaturesImage = mock(FeaturesImage.class, RETURNS_DEEP_STUBS);
// Feature disabled on start.
- when(mockedImage.features().finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 0);
- service.onNewMetadataImage(mockedImage, mock(MetadataDelta.class)); // Jobs will not execute as feature is OFF in image.
+ when(mockedFeaturesImage.finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 0);
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mockedFeaturesImage, mock(CoordinatorMetadataDelta.class)); // Jobs will not execute as feature is OFF in image.
verify(timer, times(0)).add(any()); // Timer task not added.
verify(runtime, times(0)).scheduleWriteOperation(
@@ -2090,9 +2091,9 @@ public void testPeriodicJobsDoNotRunWhenShareGroupsDisabled() throws Interrupted
assertFalse(service.shouldRunPeriodicJob());
// Enable feature.
- Mockito.reset(mockedImage);
- when(mockedImage.features().finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 1);
- service.onNewMetadataImage(mockedImage, mock(MetadataDelta.class)); // Jobs will execute as feature is ON in image.
+ Mockito.reset(mockedFeaturesImage);
+ when(mockedFeaturesImage.finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 1);
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mockedFeaturesImage, mock(CoordinatorMetadataDelta.class)); // Jobs will execute as feature is ON in image.
verify(timer, times(2)).add(any()); // Timer task added twice (prune, snapshot).
timer.advanceClock(30001L);
@@ -2110,9 +2111,9 @@ public void testPeriodicJobsDoNotRunWhenShareGroupsDisabled() throws Interrupted
assertTrue(service.shouldRunPeriodicJob());
// Disable feature
- Mockito.reset(mockedImage);
- when(mockedImage.features().finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 0);
- service.onNewMetadataImage(mockedImage, mock(MetadataDelta.class)); // Jobs will not execute as feature is on in image.
+ Mockito.reset(mockedFeaturesImage);
+ when(mockedFeaturesImage.finalizedVersions().getOrDefault(eq(ShareVersion.FEATURE_NAME), anyShort())).thenReturn((short) 0);
+ service.onNewMetadataImage(mock(CoordinatorMetadataImage.class), mockedFeaturesImage, mock(CoordinatorMetadataDelta.class)); // Jobs will not execute as feature is on in image.
timer.advanceClock(30001L);
verify(timer, times(4)).add(any()); // Tasks added but will return immediately.
diff --git a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java
index ddd0a566d6b2e..08990cd8510b9 100644
--- a/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java
+++ b/share-coordinator/src/test/java/org/apache/kafka/coordinator/share/ShareCoordinatorShardTest.java
@@ -38,19 +38,21 @@
import org.apache.kafka.common.utils.LogContext;
import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetrics;
import org.apache.kafka.coordinator.common.runtime.CoordinatorMetricsShard;
import org.apache.kafka.coordinator.common.runtime.CoordinatorRecord;
import org.apache.kafka.coordinator.common.runtime.CoordinatorResult;
+import org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotKey;
import org.apache.kafka.coordinator.share.generated.ShareSnapshotValue;
import org.apache.kafka.coordinator.share.generated.ShareUpdateKey;
import org.apache.kafka.coordinator.share.generated.ShareUpdateValue;
import org.apache.kafka.coordinator.share.metrics.ShareCoordinatorMetrics;
+import org.apache.kafka.image.ClusterImage;
import org.apache.kafka.image.MetadataImage;
import org.apache.kafka.image.TopicImage;
import org.apache.kafka.image.TopicsImage;
-import org.apache.kafka.metadata.PartitionRegistration;
import org.apache.kafka.server.common.ApiMessageAndVersion;
import org.apache.kafka.server.share.SharePartitionKey;
import org.apache.kafka.server.share.persister.PartitionFactory;
@@ -73,7 +75,6 @@
import static org.junit.jupiter.api.Assertions.assertNull;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.anyInt;
import static org.mockito.ArgumentMatchers.eq;
import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
import static org.mockito.Mockito.mock;
@@ -97,13 +98,13 @@ public static class ShareCoordinatorShardBuilder {
private final CoordinatorMetrics coordinatorMetrics = mock(CoordinatorMetrics.class);
private final CoordinatorMetricsShard metricsShard = mock(CoordinatorMetricsShard.class);
private final SnapshotRegistry snapshotRegistry = new SnapshotRegistry(logContext);
- private MetadataImage metadataImage = null;
+ private CoordinatorMetadataImage metadataImage = null;
private Map configOverrides = new HashMap<>();
ShareCoordinatorOffsetsManager offsetsManager = mock(ShareCoordinatorOffsetsManager.class);
private Time time;
ShareCoordinatorShard build() {
- if (metadataImage == null) metadataImage = mock(MetadataImage.class, RETURNS_DEEP_STUBS);
+ if (metadataImage == null) metadataImage = mock(CoordinatorMetadataImage.class, RETURNS_DEEP_STUBS);
if (config == null) {
config = ShareCoordinatorTestConfig.createConfig(ShareCoordinatorTestConfig.testConfigMap(configOverrides));
}
@@ -117,8 +118,10 @@ ShareCoordinatorShard build() {
offsetsManager,
time == null ? TIME : time
);
- when(metadataImage.topics().getTopic((Uuid) any())).thenReturn(mock(TopicImage.class));
- when(metadataImage.topics().getPartition(any(), anyInt())).thenReturn(mock(PartitionRegistration.class));
+
+ var topicMetadata = mock(CoordinatorMetadataImage.TopicMetadata.class);
+ when(topicMetadata.partitionCount()).thenReturn(PARTITION + 1);
+ when(metadataImage.topicMetadata((Uuid) any())).thenReturn(Optional.of(topicMetadata));
shard.onNewMetadataImage(metadataImage, null);
return shard;
}
@@ -1081,7 +1084,7 @@ public void testDeleteNullMetadataImage() {
@Test
public void testDeleteTopicIdNonExistentInMetadataImage() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
@@ -1112,7 +1115,8 @@ public void testDeleteTopicIdNonExistentInMetadataImage() {
@Test
public void testDeletePartitionIdNonExistentInMetadataImage() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ when(image.cluster()).thenReturn(mock(ClusterImage.class));
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
@@ -1123,12 +1127,10 @@ public void testDeletePartitionIdNonExistentInMetadataImage() {
// topic id found in cache
TopicsImage topicsImage = mock(TopicsImage.class);
- when(topicsImage.getTopic(eq(TOPIC_ID))).thenReturn(
- mock(TopicImage.class)
- );
- when(image.topics()).thenReturn(
- topicsImage
- );
+ TopicImage topicImage = mock(TopicImage.class);
+ when(topicImage.partitions()).thenReturn(Map.of());
+ when(topicsImage.getTopic(eq(TOPIC_ID))).thenReturn(topicImage);
+ when(image.topics()).thenReturn(topicsImage);
// partition id not found
when(topicsImage.getPartition(eq(TOPIC_ID), eq(0))).thenReturn(
@@ -1143,7 +1145,7 @@ public void testDeletePartitionIdNonExistentInMetadataImage() {
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
verify(topicsImage, times(1)).getTopic(eq(TOPIC_ID));
- verify(topicsImage, times(1)).getPartition(eq(TOPIC_ID), eq(0));
+ verify(topicImage, times(1)).partitions();
}
@Test
@@ -1255,7 +1257,7 @@ public void testInitializeNullMetadataImage() {
@Test
public void testInitializeTopicIdNonExistentInMetadataImage() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData()
.setGroupId(GROUP_ID)
@@ -1284,7 +1286,8 @@ public void testInitializeTopicIdNonExistentInMetadataImage() {
@Test
public void testInitializePartitionIdNonExistentInMetadataImage() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ when(image.cluster()).thenReturn(mock(ClusterImage.class));
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData()
.setGroupId(GROUP_ID)
@@ -1296,8 +1299,10 @@ public void testInitializePartitionIdNonExistentInMetadataImage() {
));
// topic id found in cache
+ TopicImage topicImage = mock(TopicImage.class);
+ when(topicImage.partitions()).thenReturn(Map.of());
TopicsImage topicsImage = mock(TopicsImage.class);
- when(topicsImage.getTopic(eq(TOPIC_ID))).thenReturn(mock(TopicImage.class));
+ when(topicsImage.getTopic(eq(TOPIC_ID))).thenReturn(topicImage);
when(image.topics()).thenReturn(topicsImage);
// partition id not found
@@ -1311,13 +1316,13 @@ public void testInitializePartitionIdNonExistentInMetadataImage() {
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
verify(topicsImage, times(1)).getTopic(eq(TOPIC_ID));
- verify(topicsImage, times(1)).getPartition(eq(TOPIC_ID), eq(0));
+ verify(topicImage, times(1)).partitions();
}
@Test
public void testSnapshotColdPartitionsNoEligiblePartitions() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int offset = 0;
int producerId = 0;
short producerEpoch = 0;
@@ -1383,7 +1388,7 @@ public void testSnapshotColdPartitionsNoEligiblePartitions() {
@Test
public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int offset = 0;
int producerId = 0;
short producerEpoch = 0;
@@ -1480,7 +1485,7 @@ public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() {
@Test
public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int offset = 0;
int producerId = 0;
short producerEpoch = 0;
@@ -1554,7 +1559,7 @@ public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() {
@Test
public void testSnapshotColdPartitionsPartialEligiblePartitions() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int offset = 0;
int producerId = 0;
short producerEpoch = 0;
@@ -1668,7 +1673,7 @@ public void testOnTopicsDeletedEmptyTopicIds() {
@Test
public void testOnTopicsDeletedTopicIds() {
MetadataImage image = mock(MetadataImage.class);
- shard.onNewMetadataImage(image, null);
+ shard.onNewMetadataImage(new KRaftCoordinatorMetadataImage(image), null);
int offset = 0;
int producerId = 0;