KAFKA-19441: encapsulate MetadataImage in GroupCoordinator/ShareCoordinator (#20061)
CI / build (push) Waiting to run Details

The MetadataImage has a lot of stuff in it and it gets passed around in
many places in the new GroupCoordinator. This makes it difficult to
understand what metadata the group coordinator actually relies on and
makes it too easy to use metadata in ways it wasn't meant to be used. 

This change encapsulate the MetadataImage in an interface
(`CoordinatorMetadataImage`) that indicates and controls what metadata
the group coordinator actually uses. Now it is much easier at a glance
to see what dependencies the GroupCoordinator has on the metadata. Also,
now we have a level of indirection that allows more flexibility in how
the GroupCoordinator is provided the metadata it needs.
This commit is contained in:
Elizabeth Bennett 2025-07-18 09:16:54 +09:00 committed by GitHub
parent f33c5e9d72
commit f81853ca88
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
70 changed files with 1597 additions and 955 deletions

View File

@ -57,6 +57,7 @@
<allow pkg="org.apache.kafka.coordinator.common" />
<allow pkg="org.apache.kafka.deferred" />
<allow pkg="org.apache.kafka.image" />
<allow pkg="org.apache.kafka.metadata" />
<allow pkg="org.apache.kafka.server.authorizer" />
<allow pkg="org.apache.kafka.server.common" />
<allow pkg="org.apache.kafka.server.metrics" />

View File

@ -52,6 +52,8 @@
<allow pkg="org.apache.kafka.server"/>
<allow pkg="org.apache.kafka.storage"/>
<allow pkg="org.apache.kafka.clients"/>
<allow class="org.apache.kafka.coordinator.common.runtime.CoordinatorMetadataImage"/>
<allow class="org.apache.kafka.coordinator.common.runtime.KRaftCoordinatorMetadataImage"/>
<allow class="org.apache.kafka.coordinator.common.runtime.HdrHistogram"/>
<allow pkg="org.apache.kafka.coordinator.group"/>
<allow pkg="org.apache.kafka.image"/>

View File

@ -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<Uuid> createdTopicIds();
Collection<Uuid> changedTopicIds();
Set<Uuid> 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<Uuid> createdTopicIds() {
return Set.of();
}
@Override
public Collection<Uuid> changedTopicIds() {
return Set.of();
}
@Override
public Set<Uuid> deletedTopicIds() {
return Set.of();
}
@Override
public CoordinatorMetadataImage image() {
return CoordinatorMetadataImage.EMPTY;
}
};
}
}

View File

@ -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<Uuid> topicIds();
Set<String> topicNames();
Optional<TopicMetadata> topicMetadata(String topicName);
Optional<TopicMetadata> topicMetadata(Uuid topicId);
CoordinatorMetadataDelta emptyDelta();
long version();
boolean isEmpty();
/**
* Metadata about a particular topic
*/
interface TopicMetadata {
String name();
Uuid id();
int partitionCount();
List<String> partitionRacks(int partitionId);
}
private static CoordinatorMetadataImage emptyImage() {
return new CoordinatorMetadataImage() {
@Override
public Set<Uuid> topicIds() {
return Set.of();
}
@Override
public Set<String> topicNames() {
return Set.of();
}
@Override
public Optional<TopicMetadata> topicMetadata(String topicName) {
return Optional.empty();
}
@Override
public Optional<TopicMetadata> 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;
}
};
}
}

View File

@ -39,8 +39,6 @@ import org.apache.kafka.common.utils.Time;
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 class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
/**
* The latest known metadata image.
*/
private volatile MetadataImage metadataImage = MetadataImage.EMPTY;
private volatile CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
/**
* Constructor.
@ -2481,18 +2479,18 @@ public class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
* @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 class CoordinatorRuntime<S extends CoordinatorShard<U>, U> implements Aut
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);
}
});
});

View File

@ -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<U> {
*
* @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

View File

@ -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<Uuid> createdTopicIds() {
if (metadataDelta == null || metadataDelta.topicsDelta() == null) {
return Set.of();
}
return metadataDelta.topicsDelta().createdTopicIds();
}
@Override
public Collection<Uuid> changedTopicIds() {
if (metadataDelta == null || metadataDelta.topicsDelta() == null) {
return Set.of();
}
return metadataDelta.topicsDelta().changedTopics().keySet();
}
@Override
public Set<Uuid> 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();
}
}

View File

@ -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<Uuid> topicIds() {
return Collections.unmodifiableSet(metadataImage.topics().topicsById().keySet());
}
@Override
public Set<String> topicNames() {
return Collections.unmodifiableSet(metadataImage.topics().topicsByName().keySet());
}
@Override
public Optional<TopicMetadata> 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> 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<String> partitionRacks(int partition) {
List<String> 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();
}
}
}
}

View File

@ -19,7 +19,6 @@ package org.apache.kafka.coordinator.common.runtime;
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 class SnapshottableCoordinator<S extends CoordinatorShard<U>, U> implemen
*
* @param newImage The metadata image.
*/
synchronized void onLoaded(MetadataImage newImage) {
synchronized void onLoaded(CoordinatorMetadataImage newImage) {
this.coordinator.onLoaded(newImage);
}
@ -207,7 +206,7 @@ public class SnapshottableCoordinator<S extends CoordinatorShard<U>, U> implemen
* @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);
}

View File

@ -32,7 +32,6 @@ import org.apache.kafka.common.utils.LogContext;
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 class CoordinatorRuntimeTest {
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 class CoordinatorRuntimeTest {
// 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.

View File

@ -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<Uuid> createdTopicIds = coordinatorDelta.createdTopicIds();
assertNotNull(createdTopicIds);
assertEquals(2, createdTopicIds.size());
assertTrue(createdTopicIds.contains(topicId));
assertTrue(createdTopicIds.contains(topicId2));
// deleted topics
Set<Uuid> deletedTopicIds = coordinatorDelta.deletedTopicIds();
assertNotNull(deletedTopicIds);
assertEquals(1, deletedTopicIds.size());
assertTrue(deletedTopicIds.contains(deletedTopicId));
// changed topics (also includes created topics)
Collection<Uuid> 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());
}
}

View File

@ -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<String> racks0 = topicMetadata.partitionRacks(0);
List<String> 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<String> racks0 = topicMetadata.partitionRacks(0);
List<String> 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<String> 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());
}
}

View File

@ -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 class MetadataImageBuilder {
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));
}
}

View File

@ -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)

View File

@ -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

View File

@ -53,9 +53,9 @@ import org.apache.kafka.common.message.TxnOffsetCommitRequestData;
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 @@ public interface GroupCoordinator {
* @param delta The metadata delta.
*/
void onNewMetadataImage(
MetadataImage newImage,
MetadataDelta delta
CoordinatorMetadataImage newImage,
CoordinatorMetadataDelta delta
);
/**

View File

@ -84,6 +84,8 @@ import org.apache.kafka.common.utils.Time;
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.common.runtime.PartitionWriter;
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 class GroupCoordinatorService implements GroupCoordinator {
/**
* 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 class GroupCoordinatorService implements GroupCoordinator {
List<ReadShareGroupStateSummaryRequestData.ReadStateSummaryData> readStateSummaryData = new ArrayList<>(requestData.topics().size());
List<DescribeShareGroupOffsetsResponseData.DescribeShareGroupOffsetsResponseTopic> describeShareGroupOffsetsResponseTopicList = new ArrayList<>(requestData.topics().size());
requestData.topics().forEach(topic -> {
Uuid topicId = metadataImage.topics().topicNameToIdView().get(topic.topicName());
if (topicId != null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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 class GroupCoordinatorService implements GroupCoordinator {
ReadShareGroupStateSummaryRequestData readSummaryRequestData = new ReadShareGroupStateSummaryRequestData()
.setGroupId(requestData.groupId());
topicPartitionMap.forEach((topicId, partitionSet) -> {
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 class GroupCoordinatorService implements GroupCoordinator {
partitionIndex -> new ReadShareGroupStateSummaryRequestData.PartitionData().setPartition(partitionIndex)
).toList()
));
}
});
});
return readShareGroupStateSummary(readSummaryRequestData, requestTopicIdToNameMapping, describeShareGroupOffsetsResponseTopicList);
});
@ -1917,18 +1916,20 @@ public class GroupCoordinatorService implements GroupCoordinator {
.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 class GroupCoordinatorService implements GroupCoordinator {
// 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<Uuid> topicIds = new HashSet<>();
for (TopicPartition tp : topicPartitions) {
TopicImage image = metadataImage.topics().getTopic(tp.topic());
if (image != null) {
topicIds.add(image.id());
}
}
Set<Uuid> 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 class GroupCoordinatorService implements GroupCoordinator {
}
/**
* 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;

View File

@ -61,6 +61,8 @@ import org.apache.kafka.common.requests.TransactionResult;
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.GroupCoordinatorMetrics;
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 @@ public class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
* @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 class GroupCoordinatorShard implements CoordinatorShard<CoordinatorRecord
* @param delta The delta image.
*/
@Override
public void onNewMetadataImage(MetadataImage newImage, MetadataDelta delta) {
public void onNewMetadataImage(CoordinatorMetadataImage newImage, CoordinatorMetadataDelta delta) {
groupMetadataManager.onNewMetadataImage(newImage, delta);
}

View File

@ -80,6 +80,8 @@ import org.apache.kafka.common.resource.ResourcePattern;
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.ConfiguredTopology;
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 class GroupMetadataManager {
private CoordinatorExecutor<CoordinatorRecord> 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<Plugin<Authorizer>> authorizerPlugin = null;
@ -340,7 +337,7 @@ public class GroupMetadataManager {
return this;
}
Builder withMetadataImage(MetadataImage metadataImage) {
Builder withMetadataImage(CoordinatorMetadataImage metadataImage) {
this.metadataImage = metadataImage;
return this;
}
@ -363,7 +360,7 @@ public class GroupMetadataManager {
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 @@ public class GroupMetadataManager {
/**
* 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<String, Long> 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 @@ public class GroupMetadataManager {
CoordinatorTimer<Void, CoordinatorRecord> timer,
CoordinatorExecutor<CoordinatorRecord> executor,
GroupCoordinatorMetricsShard metrics,
MetadataImage metadataImage,
CoordinatorMetadataImage metadataImage,
GroupCoordinatorConfig config,
GroupConfigManager groupConfigManager,
ShareGroupPartitionAssignor shareGroupAssignor,
@ -560,7 +557,7 @@ public class 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 class GroupMetadataManager {
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 class GroupMetadataManager {
describedGroups.add(shareGroup(groupId, committedOffset).asDescribedGroup(
committedOffset,
shareGroupAssignor.name(),
metadataImage.topics()
metadataImage
));
} catch (GroupIdNotFoundException exception) {
describedGroups.add(new ShareGroupDescribeResponseData.DescribedGroup()
@ -1940,7 +1937,7 @@ public class GroupMetadataManager {
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 @@ public class GroupMetadataManager {
group::currentPartitionEpoch,
targetAssignmentEpoch,
targetAssignment,
toTopicPartitions(subscription.ownedPartitions(), metadataImage.topics()),
toTopicPartitions(subscription.ownedPartitions(), metadataImage),
records
);
@ -2748,18 +2745,19 @@ public class GroupMetadataManager {
// 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<Integer> alreadyInitializedPartSet = alreadyInitialized.containsKey(topicImage.id()) ? alreadyInitialized.get(topicImage.id()).partitions() : Set.of();
if (alreadyInitializedPartSet.isEmpty() || alreadyInitializedPartSet.size() < topicImage.partitions().size()) {
Set<Integer> partitionSet = new HashSet<>(topicImage.partitions().keySet());
partitionSet.removeAll(alreadyInitializedPartSet);
metadataImage.topicMetadata(topicName).ifPresent(topicMetadata -> {
Set<Integer> 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<Integer> 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 @@ public class GroupMetadataManager {
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 class GroupMetadataManager {
String groupId,
Logger log,
Time time,
MetadataImage image,
CoordinatorMetadataImage image,
Optional<Plugin<Authorizer>> authorizerPlugin,
Set<String> regexes
) {
@ -3235,7 +3234,7 @@ public class GroupMetadataManager {
}
}
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 class GroupMetadataManager {
resolvedRegexes
);
long version = image.provenance().lastContainedOffset();
long version = image.version();
Map<String, ResolvedRegularExpression> result = new HashMap<>(resolvedRegexes.size());
for (Map.Entry<String, Set<String>> resolvedRegex : resolvedRegexes.entrySet()) {
result.put(
@ -3259,7 +3258,7 @@ public class GroupMetadataManager {
}
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 @@ public class GroupMetadataManager {
int groupEpoch,
StreamsGroupMember updatedMember,
ConfiguredTopology configuredTopology,
MetadataImage metadataImage,
CoordinatorMetadataImage metadataImage,
List<CoordinatorRecord> records
) {
TaskAssignor assignor = streamsGroupAssignor(group.groupId());
@ -4925,24 +4924,20 @@ public class GroupMetadataManager {
}
private Map<Uuid, String> attachTopicName(Set<Uuid> topicIds) {
TopicsImage topicsImage = metadataImage.topics();
Map<Uuid, String> finalMap = new HashMap<>();
for (Uuid topicId : topicIds) {
TopicImage topicImage = topicsImage.getTopic(topicId);
String topicName = (topicImage != null) ? topicImage.name() : "<UNKNOWN>";
String topicName = metadataImage.topicMetadata(topicId).map(CoordinatorMetadataImage.TopicMetadata::name).orElse("<UNKNOWN>");
finalMap.put(topicId, topicName);
}
return Collections.unmodifiableMap(finalMap);
}
private Map<Uuid, InitMapValue> attachInitValue(Map<Uuid, Set<Integer>> initMap) {
TopicsImage topicsImage = metadataImage.topics();
Map<Uuid, InitMapValue> finalMap = new HashMap<>();
long timestamp = time.milliseconds();
for (Map.Entry<Uuid, Set<Integer>> entry : initMap.entrySet()) {
Uuid topicId = entry.getKey();
TopicImage topicImage = topicsImage.getTopic(topicId);
String topicName = (topicImage != null) ? topicImage.name() : "<UNKNOWN>";
String topicName = metadataImage.topicMetadata(topicId).map(CoordinatorMetadataImage.TopicMetadata::name).orElse("<UNKNOWN>");
finalMap.put(topicId, new InitMapValue(topicName, entry.getValue(), timestamp));
}
return Collections.unmodifiableMap(finalMap);
@ -5774,41 +5769,37 @@ public class GroupMetadataManager {
* @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<String> 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 @@ public class GroupMetadataManager {
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 class GroupMetadataManager {
// 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<Uuid> currentDeleting = shareGroupStatePartitionMetadata.get(shareGroupId).deletingTopics();
Map<Uuid, InitMapValue> deleteRetryCandidates = new HashMap<>();
Set<Uuid> 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<CoordinatorMetadataImage.TopicMetadata> 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 class GroupMetadataManager {
Set<Uuid> deletingTopics = new HashSet<>(currentMap.deletingTopics());
requestData.topics().forEach(topic -> {
TopicImage topicImage = metadataImage.topics().getTopic(topic.topicName());
if (topicImage != null) {
Uuid topicId = topicImage.id();
Optional<CoordinatorMetadataImage.TopicMetadata> 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 class GroupMetadataManager {
// 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<DeleteShareGroupStateRequestData.PartitionData> 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 class GroupMetadataManager {
Map<Uuid, Map<Integer, Long>> offsetByTopicPartitions = new HashMap<>();
alterShareGroupOffsetsRequest.topics().forEach(topic -> {
TopicImage topicImage = metadataImage.topics().getTopic(topic.topicName());
if (topicImage != null) {
Uuid topicId = topicImage.id();
Set<Integer> existingPartitions = new HashSet<>(topicImage.partitions().keySet());
Optional<CoordinatorMetadataImage.TopicMetadata> topicMetadataOpt = metadataImage.topicMetadata(topic.topicName());
if (topicMetadataOpt.isPresent()) {
var topicMetadata = topicMetadataOpt.get();
Uuid topicId = topicMetadata.id();
List<AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition> 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 class GroupMetadataManager {
topic.topicName(),
topic.partitions().stream()
.map(AlterShareGroupOffsetsRequestData.AlterShareGroupOffsetsRequestPartition::partitionIndex)
.filter(existingPartitions::contains)
.filter(part -> part < topicMetadata.partitionCount())
.collect(Collectors.toSet()),
currentTimeMs
));

View File

@ -23,13 +23,9 @@ import org.apache.kafka.common.message.ConsumerGroupHeartbeatRequestData;
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 class Utils {
* 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<Uuid, Set<Integer>> 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 class Utils {
/**
* 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<Uuid, Set<Integer>> toTopicPartitionMap(
ConsumerProtocolAssignment consumerProtocolAssignment,
TopicsImage topicsImage
CoordinatorMetadataImage metadataImage
) {
Map<Uuid, Set<Integer>> 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 class Utils {
/**
* 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<ConsumerGroupHeartbeatRequestData.TopicPartitions> toTopicPartitions(
ConsumerProtocolSubscription.TopicPartitionCollection topicPartitionCollection,
TopicsImage topicsImage
CoordinatorMetadataImage metadataImage
) {
List<ConsumerGroupHeartbeatRequestData.TopicPartitions> 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 class Utils {
* 5. For each partition, write the partition ID and a sorted list of rack identifiers.
* - Rack identifiers are formatted as "<length1><value1><length2><value2>" 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<CoordinatorMetadataImage.TopicMetadata> 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());
.putLong(topicMetadata.id().getMostSignificantBits())
.putLong(topicMetadata.id().getLeastSignificantBits())
.putString(topicMetadata.name())
.putInt(topicMetadata.partitionCount());
ClusterImage clusterImage = metadataImage.cluster();
List<String> racks = new ArrayList<>();
for (int i = 0; i < topicImage.partitions().size(); i++) {
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<String> partitionRacks = topicMetadata.partitionRacks(i);
Collections.sort(partitionRacks);
Collections.sort(racks);
for (String rack : racks) {
for (String rack : partitionRacks) {
// Format: "<length><value>"
// 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 ",,".

View File

@ -33,6 +33,7 @@ import org.apache.kafka.common.protocol.types.SchemaException;
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.OffsetExpirationCondition;
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 class ClassicGroup implements Group {
/**
* 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 class ClassicGroup implements Group {
ConsumerGroupMember joiningMember,
LogContext logContext,
Time time,
MetadataImage metadataImage
CoordinatorMetadataImage image
) {
ClassicGroup classicGroup = new ClassicGroup(
logContext,
@ -1427,7 +1427,7 @@ public class ClassicGroup implements Group {
byte[] assignment = Utils.toArray(ConsumerProtocol.serializeAssignment(
toConsumerProtocolAssignment(
consumerGroup.targetAssignment().get(memberId).partitions(),
metadataImage.topics()
image
),
ConsumerProtocol.deserializeVersion(
ByteBuffer.wrap(classicGroupMember.metadata(classicGroup.protocolName().orElse("")))

View File

@ -19,11 +19,10 @@ package org.apache.kafka.coordinator.group.modern;
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.HashSet;
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 abstract class ModernGroup<T extends ModernGroupMember> implements Group
public static long computeMetadataHash(
Map<String, SubscriptionCount> subscribedTopicNames,
Map<String, Long> topicHashCache,
MetadataImage metadataImage
CoordinatorMetadataImage metadataImage
) {
Map<String, Long> 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<String, Long> 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);
}

View File

@ -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 class SubscribedTopicDescriberImpl implements SubscribedTopicDescriber {
*/
@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 class SubscribedTopicDescriberImpl implements SubscribedTopicDescriber {
*/
@Override
public Set<String> racksForPartition(Uuid topicId, int partition) {
TopicImage topic = metadataImage.topics().getTopic(topicId);
if (topic != null) {
PartitionRegistration partitionRegistration = topic.partitions().get(partition);
if (partitionRegistration != null) {
Set<String> 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<CoordinatorMetadataImage.TopicMetadata> topicMetadataOp = metadataImage.topicMetadata(topicId);
if (topicMetadataOp.isEmpty()) {
return Set.of();
}
CoordinatorMetadataImage.TopicMetadata topicMetadata = topicMetadataOp.get();
List<String> racks = topicMetadata.partitionRacks(partition);
if (racks == null) {
return Set.of();
} else {
return new HashSet<>(racks);
}
return Set.of();
}
@Override

View File

@ -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.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.coordinator.group.modern.share.ShareGroupMember;
import org.apache.kafka.image.MetadataImage;
import java.util.ArrayList;
import java.util.Collections;
@ -269,7 +269,7 @@ public abstract class TargetAssignmentBuilder<T extends ModernGroupMember, U ext
/**
* 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 abstract class TargetAssignmentBuilder<T extends ModernGroupMember, U ext
* @return This object.
*/
public U withMetadataImage(
MetadataImage metadataImage
CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return self();
@ -427,7 +427,7 @@ public abstract class TargetAssignmentBuilder<T extends ModernGroupMember, U ext
*/
public TargetAssignmentResult build() throws PartitionAssignorException {
Map<String, MemberSubscriptionAndAssignmentImpl> 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) ->

View File

@ -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 class TopicIds implements Set<Uuid> {
/**
* @return The TopicsImage used by the resolver.
*/
TopicsImage image();
CoordinatorMetadataImage image();
/**
* Converts a topic id to a topic name.
@ -70,31 +69,27 @@ public class TopicIds implements Set<Uuid> {
* 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 class TopicIds implements Set<Uuid> {
* TargetAssignmentBuilder.build() call.
*/
public static class CachedTopicResolver implements TopicResolver {
private final TopicsImage image;
private final CoordinatorMetadataImage image;
private final Map<String, Uuid> topicIds = new HashMap<>();
private final Map<Uuid, String> 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 class TopicIds implements Set<Uuid> {
public TopicIds(
Set<String> topicNames,
TopicsImage image
CoordinatorMetadataImage image
) {
this.topicNames = Objects.requireNonNull(topicNames);
this.resolver = new DefaultTopicResolver(image);

View File

@ -29,6 +29,7 @@ import org.apache.kafka.common.message.ConsumerProtocolSubscription;
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.MemberState;
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 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
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 class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
entry -> describedGroup.members().add(
entry.getValue().asConsumerGroupDescribeMember(
targetAssignment.get(entry.getValue().memberId(), committedOffset),
topicsImage
image
)
)
);
@ -1145,7 +1144,7 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
GroupCoordinatorMetricsShard metrics,
ClassicGroup classicGroup,
Map<String, Long> topicHashCache,
MetadataImage metadataImage
CoordinatorMetadataImage metadataImage
) {
String groupId = classicGroup.groupId();
ConsumerGroup consumerGroup = new ConsumerGroup(snapshotRegistry, groupId, metrics);
@ -1165,7 +1164,7 @@ public class ConsumerGroup extends ModernGroup<ConsumerGroupMember> {
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,

View File

@ -19,14 +19,13 @@ package org.apache.kafka.coordinator.group.modern.consumer;
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 class ConsumerGroupMember extends ModernGroupMember {
/**
* @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 class ConsumerGroupMember extends ModernGroupMember {
private static List<ConsumerGroupDescribeResponseData.TopicPartitions> topicPartitionsFromMap(
Map<Uuid, Set<Integer>> partitions,
TopicsImage topicsImage
CoordinatorMetadataImage image
) {
List<ConsumerGroupDescribeResponseData.TopicPartitions> 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;
}

View File

@ -22,11 +22,11 @@ import org.apache.kafka.common.errors.GroupIdNotFoundException;
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 @@ public class ShareGroup extends ModernGroup<ShareGroupMember> {
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 class ShareGroup extends ModernGroup<ShareGroupMember> {
members.entrySet(committedOffset).forEach(
entry -> describedGroup.members().add(
entry.getValue().asShareGroupDescribeMember(
topicsImage
image
)
)
);

View File

@ -18,13 +18,12 @@ package org.apache.kafka.coordinator.group.modern.share;
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 @@ public class ShareGroupMember extends ModernGroupMember {
/**
* 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 class ShareGroupMember extends ModernGroupMember {
private static List<ShareGroupDescribeResponseData.TopicPartitions> topicPartitionsFromMap(
Map<Uuid, Set<Integer>> partitions,
TopicsImage topicsImage
CoordinatorMetadataImage image
) {
List<ShareGroupDescribeResponseData.TopicPartitions> 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;
}

View File

@ -25,6 +25,7 @@ import org.apache.kafka.common.message.StreamsGroupDescribeResponseData;
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.Utils;
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 class StreamsGroup implements Group {
* @return The metadata hash.
*/
public long computeMetadataHash(
MetadataImage metadataImage,
CoordinatorMetadataImage metadataImage,
Map<String, Long> topicHashCache,
StreamsTopology topology
) {
@ -626,13 +625,11 @@ public class StreamsGroup implements Group {
Map<String, Long> 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);
}

View File

@ -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.MemberAssignment;
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 class TargetAssignmentBuilder {
* @return This object.
*/
public TargetAssignmentBuilder withMetadataImage(
MetadataImage metadataImage
CoordinatorMetadataImage metadataImage
) {
this.metadataImage = metadataImage;
return this;

View File

@ -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 @@ import java.util.SortedMap;
* @param metadataImage The metadata image
* @param subtopologyMap The configured subtopologies
*/
public record TopologyMetadata(MetadataImage metadataImage, SortedMap<String, ConfiguredSubtopology> subtopologyMap) implements TopologyDescriber {
public record TopologyMetadata(CoordinatorMetadataImage metadataImage, SortedMap<String, ConfiguredSubtopology> subtopologyMap) implements TopologyDescriber {
public TopologyMetadata {
metadataImage = Objects.requireNonNull(metadataImage);

View File

@ -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 @@ public class 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<String, Set<Integer>> activeTasks = streamsGroupMember.assignedTasks().activeTasks();
Map<String, Set<Integer>> standbyTasks = streamsGroupMember.assignedTasks().standbyTasks();
@ -53,7 +53,7 @@ public class EndpointToPartitionsManager {
private static List<StreamsGroupHeartbeatResponseData.TopicPartition> topicPartitions(final Map<String, Set<Integer>> tasks,
final Map<String, ConfiguredSubtopology> configuredSubtopologies,
final MetadataImage metadataImage) {
final CoordinatorMetadataImage metadataImage) {
List<StreamsGroupHeartbeatResponseData.TopicPartition> topicPartitionsForTasks = new ArrayList<>();
for (Map.Entry<String, Set<Integer>> taskEntry : tasks.entrySet()) {
String subtopologyId = taskEntry.getKey();
@ -70,13 +70,13 @@ public class EndpointToPartitionsManager {
private static List<StreamsGroupHeartbeatResponseData.TopicPartition> topicPartitionListForTask(final Set<Integer> taskSet,
final Set<String> topicNames,
final MetadataImage metadataImage) {
final CoordinatorMetadataImage metadataImage) {
return topicNames.stream().map(topic -> {
TopicImage topicImage = metadataImage.topics().getTopic(topic);
if (topicImage == null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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<Integer> tpPartitions = new ArrayList<>(taskSet);

View File

@ -20,10 +20,9 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopic;
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 @@ import java.util.stream.Stream;
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<StreamsGroupTopologyValue.Subtopology> subtopologies = topology.subtopologies().values();
@ -73,23 +72,23 @@ public class InternalTopicManager {
try {
Optional<TopicConfigurationException> topicConfigurationException = Optional.empty();
throwOnMissingSourceTopics(topology, topicsImage);
throwOnMissingSourceTopics(topology, metadataImage);
Map<String, Integer> decidedPartitionCountsForInternalTopics =
decidePartitionCounts(logContext, topology, topicsImage, copartitionGroupsBySubtopology, log);
decidePartitionCounts(logContext, topology, metadataImage, copartitionGroupsBySubtopology, log);
final SortedMap<String, ConfiguredSubtopology> 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<String, CreatableTopic> internalTopicsToCreate = missingInternalTopics(configuredSubtopologies, topology, topicsImage);
Map<String, CreatableTopic> internalTopicsToCreate = missingInternalTopics(configuredSubtopologies, topology, metadataImage);
if (!internalTopicsToCreate.isEmpty()) {
topicConfigurationException = Optional.of(TopicConfigurationException.missingInternalTopics(
"Internal topics are missing: " + internalTopicsToCreate.keySet()
@ -122,11 +121,11 @@ public class InternalTopicManager {
}
private static void throwOnMissingSourceTopics(final StreamsTopology topology,
final TopicsImage topicsImage) {
final CoordinatorMetadataImage metadataImage) {
TreeSet<String> 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 @@ public class InternalTopicManager {
private static Map<String, Integer> decidePartitionCounts(final LogContext logContext,
final StreamsTopology topology,
final TopicsImage topicsImage,
final CoordinatorMetadataImage metadataImage,
final Map<String, Collection<Set<String>>> copartitionGroupsBySubtopology,
final Logger log) {
final Map<String, Integer> decidedPartitionCountsForInternalTopics = new HashMap<>();
final Function<String, OptionalInt> topicPartitionCountProvider =
topic -> getPartitionCount(topicsImage, topic, decidedPartitionCountsForInternalTopics);
topic -> getPartitionCount(metadataImage, topic, decidedPartitionCountsForInternalTopics);
final RepartitionTopics repartitionTopics = new RepartitionTopics(
logContext,
topology.subtopologies().values(),
@ -196,7 +195,7 @@ public class InternalTopicManager {
private static Map<String, CreatableTopic> missingInternalTopics(Map<String, ConfiguredSubtopology> subtopologyMap,
StreamsTopology topology,
TopicsImage topicsImage) {
CoordinatorMetadataImage metadataImage) {
final Map<String, CreatableTopic> topicsToCreate = new HashMap<>();
for (ConfiguredSubtopology subtopology : subtopologyMap.values()) {
@ -206,33 +205,31 @@ public class InternalTopicManager {
.forEach(x -> 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<String, Integer> decidedPartitionCountsForInternalTopics) {
final TopicImage topicImage = topicsImage.getTopic(topic);
if (topicImage == null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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 @@ public class InternalTopicManager {
}
private static ConfiguredSubtopology fromPersistedSubtopology(final StreamsGroupTopologyValue.Subtopology subtopology,
final TopicsImage topicsImage,
final CoordinatorMetadataImage metadataImage,
final Map<String, Integer> 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 @@ public class InternalTopicManager {
}
private static int computeNumberOfTasks(final StreamsGroupTopologyValue.Subtopology subtopology,
final TopicsImage topicsImage,
final CoordinatorMetadataImage metadataImage,
final Map<String, Integer> 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(

View File

@ -87,13 +87,16 @@ import org.apache.kafka.common.security.auth.SecurityProtocol;
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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 class GroupCoordinatorServiceTest {
.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 @@ public class GroupCoordinatorServiceTest {
private CoordinatorRuntime<GroupCoordinatorShard, CoordinatorRecord> 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 @@ public class GroupCoordinatorServiceTest {
GroupCoordinatorService build(boolean serviceStartup) {
if (metadataImage == null) {
metadataImage = mock(MetadataImage.class);
metadataImage = mock(CoordinatorMetadataImage.class);
}
GroupCoordinatorService service = new GroupCoordinatorService(
@ -5671,9 +5674,12 @@ public class GroupCoordinatorServiceTest {
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;
}

View File

@ -42,6 +42,7 @@ import org.apache.kafka.common.requests.TransactionResult;
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.metrics.GroupCoordinatorMetricsShard;
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 class GroupCoordinatorShardTest {
@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 class GroupCoordinatorShardTest {
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 class GroupCoordinatorShardTest {
coordinatorMetrics,
metricsShard
);
coordinator.onLoaded(MetadataImage.EMPTY);
coordinator.onLoaded(CoordinatorMetadataImage.EMPTY);
// The counter is scheduled.
assertEquals(

View File

@ -53,6 +53,7 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal;
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.StreamsGroupMember;
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 class GroupMetadataManagerTestContext {
private final MockCoordinatorExecutor<CoordinatorRecord> 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<ConsumerGroupBuilder> consumerGroupBuilders = new ArrayList<>();
private final List<StreamsGroupBuilder> streamsGroupBuilders = new ArrayList<>();
@ -478,7 +478,7 @@ public class GroupMetadataManagerTestContext {
return this;
}
public Builder withMetadataImage(MetadataImage metadataImage) {
public Builder withMetadataImage(CoordinatorMetadataImage metadataImage) {
this.metadataImage = metadataImage;
return this;
}
@ -519,7 +519,7 @@ public class GroupMetadataManagerTestContext {
}
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 class GroupMetadataManagerTestContext {
);
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 class GroupMetadataManagerTestContext {
new LogContext(),
0,
group.topology().get(),
metadataImage.topics())
metadataImage)
);
}
});

View File

@ -49,6 +49,7 @@ 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.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 @@ public class OffsetMetadataManagerTest {
.withExecutor(executor)
.withSnapshotRegistry(snapshotRegistry)
.withLogContext(logContext)
.withMetadataImage(metadataImage)
.withMetadataImage(new KRaftCoordinatorMetadataImage(metadataImage))
.withGroupCoordinatorMetricsShard(metrics)
.withGroupConfigManager(configManager)
.withConfig(GroupCoordinatorConfig.fromProps(Map.of()))

View File

@ -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 @@ public class UtilsTest {
@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 @@ public class UtilsTest {
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()
)
);
}

View File

@ -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 class OptimizedUniformAssignmentBuilderTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -99,7 +100,7 @@ public class OptimizedUniformAssignmentBuilderTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -159,7 +160,7 @@ public class OptimizedUniformAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -218,7 +219,7 @@ public class OptimizedUniformAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -257,7 +258,7 @@ public class OptimizedUniformAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -313,7 +314,7 @@ public class OptimizedUniformAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -371,7 +372,7 @@ public class OptimizedUniformAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -438,7 +439,7 @@ public class OptimizedUniformAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -497,7 +498,7 @@ public class OptimizedUniformAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -553,7 +554,7 @@ public class OptimizedUniformAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -620,7 +621,7 @@ public class OptimizedUniformAssignmentBuilderTest {
HOMOGENEOUS,
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(metadataImage);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(new KRaftCoordinatorMetadataImage(metadataImage));
GroupAssignment computedAssignment = assignor.assign(
groupSpec,

View File

@ -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<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -100,7 +102,7 @@ public class RangeAssignorTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -152,7 +154,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -210,7 +212,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -270,7 +272,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -301,7 +303,7 @@ public class RangeAssignorTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = new TreeMap<>();
@ -367,7 +369,7 @@ public class RangeAssignorTest {
.addTopic(topic1Uuid, topic1Name, 5)
.build();
SubscribedTopicDescriber subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
// Initialize members with instance Ids.
@ -484,7 +486,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -543,7 +545,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -607,7 +609,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -676,7 +678,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -727,7 +729,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -791,7 +793,7 @@ public class RangeAssignorTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(

View File

@ -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 class SimpleAssignorTest {
@Test
public void testAssignWithEmptyMembers() {
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
MetadataImage.EMPTY
CoordinatorMetadataImage.EMPTY
);
GroupSpec groupSpec = new GroupSpecImpl(
@ -104,7 +106,7 @@ public class SimpleAssignorTest {
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -137,7 +139,7 @@ public class SimpleAssignorTest {
.addTopic(TOPIC_1_UUID, TOPIC_1_NAME, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = Map.of(
@ -193,7 +195,7 @@ public class SimpleAssignorTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -243,7 +245,7 @@ public class SimpleAssignorTest {
)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -291,7 +293,7 @@ public class SimpleAssignorTest {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -345,7 +347,7 @@ public class SimpleAssignorTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -408,7 +410,7 @@ public class SimpleAssignorTest {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -470,7 +472,7 @@ public class SimpleAssignorTest {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -524,7 +526,7 @@ public class SimpleAssignorTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -553,7 +555,7 @@ public class SimpleAssignorTest {
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Set<Uuid> topicsSubscription = new LinkedHashSet<>();
@ -600,7 +602,7 @@ public class SimpleAssignorTest {
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Set<Uuid> topicsSubscription = new LinkedHashSet<>();
@ -705,7 +707,7 @@ public class SimpleAssignorTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata1 = new SubscribedTopicDescriberImpl(
metadataImage1
new KRaftCoordinatorMetadataImage(metadataImage1)
);
GroupAssignment computedAssignment1 = assignor.assign(
@ -759,7 +761,7 @@ public class SimpleAssignorTest {
);
SubscribedTopicDescriberImpl subscribedTopicMetadata2 = new SubscribedTopicDescriberImpl(
metadataImage2
new KRaftCoordinatorMetadataImage(metadataImage2)
);
GroupAssignment computedAssignment2 = assignor.assign(
@ -775,11 +777,11 @@ public class SimpleAssignorTest {
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 class SimpleAssignorTest {
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

View File

@ -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 class UniformHeterogeneousAssignmentBuilderTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = new TreeMap<>();
@ -131,7 +132,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
.addTopic(topic1Uuid, topic1Name, 3)
.build();
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
Map<String, MemberSubscriptionAndAssignmentImpl> members = new TreeMap<>();
@ -188,7 +189,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -244,7 +245,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -313,7 +314,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -377,7 +378,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -442,7 +443,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -502,7 +503,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -558,7 +559,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -617,7 +618,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
invertedTargetAssignment(members)
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(
@ -666,7 +667,7 @@ public class UniformHeterogeneousAssignmentBuilderTest {
Map.of()
);
SubscribedTopicDescriberImpl subscribedTopicMetadata = new SubscribedTopicDescriberImpl(
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
GroupAssignment computedAssignment = assignor.assign(

View File

@ -39,7 +39,8 @@ import org.apache.kafka.common.utils.MockTime;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
import org.apache.kafka.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 class ClassicGroupTest {
newMember2,
logContext,
time,
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
ClassicGroup expectedClassicGroup = new ClassicGroup(
@ -1592,7 +1593,7 @@ public class ClassicGroupTest {
null,
logContext,
time,
metadataImage
new KRaftCoordinatorMetadataImage(metadataImage)
);
ClassicGroup expectedClassicGroup = new ClassicGroup(

View File

@ -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 @@ import static org.junit.jupiter.api.Assertions.assertThrows;
public class SubscribedTopicMetadataTest {
private SubscribedTopicDescriberImpl subscribedTopicMetadata;
private MetadataImage metadataImage;
private CoordinatorMetadataImage metadataImage;
private final int numPartitions = 5;
@BeforeEach
@ -43,7 +43,7 @@ public class SubscribedTopicMetadataTest {
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 class SubscribedTopicMetadataTest {
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 class SubscribedTopicMetadataTest {
// 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 class SubscribedTopicMetadataTest {
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);
}
}

View File

@ -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 @@ public class TargetAssignmentBuilderTest {
}
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<String, MemberSubscriptionAndAssignmentImpl> memberSubscriptions = new HashMap<>();
@ -251,7 +252,7 @@ public class TargetAssignmentBuilderTest {
});
// 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 class TargetAssignmentBuilderTest {
.withSubscriptionType(subscriptionType)
.withTargetAssignment(targetAssignment)
.withInvertedTargetAssignment(invertedTargetAssignment)
.withMetadataImage(metadataImage)
.withMetadataImage(cooridnatorMetadataImage)
.withResolvedRegularExpressions(resolvedRegularExpressions);
// Add the updated members or delete the deleted members.

View File

@ -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 class TopicIdsTest {
@Test
public void testSize() {
Set<String> topicNames = Set.of("foo", "bar", "baz");
Set<Uuid> topicIds = new TopicIds(topicNames, TopicsImage.EMPTY);
Set<Uuid> topicIds = new TopicIds(topicNames, CoordinatorMetadataImage.EMPTY);
assertEquals(topicNames.size(), topicIds.size());
}
@Test
public void testIsEmpty() {
Set<String> topicNames = Set.of();
Set<Uuid> topicIds = new TopicIds(topicNames, TopicsImage.EMPTY);
Set<Uuid> topicIds = new TopicIds(topicNames, CoordinatorMetadataImage.EMPTY);
assertEquals(topicNames.size(), topicIds.size());
}
@ -67,14 +68,13 @@ public class TopicIdsTest {
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<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz"), topicsImage);
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@ -88,15 +88,14 @@ public class TopicIdsTest {
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<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@ -112,14 +111,13 @@ public class TopicIdsTest {
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<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
assertTrue(topicIds.contains(fooUuid));
assertTrue(topicIds.contains(barUuid));
@ -134,15 +132,14 @@ public class TopicIdsTest {
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<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), topicsImage);
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "qux"), metadataImage);
Set<Uuid> expectedIds = Set.of(fooUuid, barUuid, bazUuid, quxUuid);
Set<Uuid> actualIds = new HashSet<>(topicIds);
@ -157,15 +154,14 @@ public class TopicIdsTest {
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<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "quux"), topicsImage);
Set<Uuid> topicIds = new TopicIds(Set.of("foo", "bar", "baz", "quux"), metadataImage);
Set<Uuid> expectedIds = Set.of(fooUuid, barUuid, bazUuid);
Set<Uuid> actualIds = new HashSet<>(topicIds);
@ -175,19 +171,12 @@ public class TopicIdsTest {
@Test
public void testEquals() {
Uuid topicId = Uuid.randomUuid();
TopicIds topicIds1 = 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 topicIds2 = new TopicIds(Set.of("topic"),
new MetadataImageBuilder()
.addTopic(topicId, "topicId", 3)
.build()
.topics()
);
TopicIds topicIds1 = new TopicIds(Set.of("topic"), metadataImage);
TopicIds topicIds2 = new TopicIds(Set.of("topic"), metadataImage);
assertEquals(topicIds1, topicIds2);
}

View File

@ -19,7 +19,8 @@ package org.apache.kafka.coordinator.group.modern.consumer;
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 class ConsumerGroupMemberTest {
.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 class ConsumerGroupMemberTest {
.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 class ConsumerGroupMemberTest {
.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);
}

View File

@ -33,10 +33,12 @@ 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.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.Assignment;
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 class ConsumerGroupTest {
.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 class ConsumerGroupTest {
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 class ConsumerGroupTest {
@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<String, Long> cache = new HashMap<>();
assertEquals(
computeGroupHash(Map.of(
@ -2084,11 +2085,11 @@ public class ConsumerGroupTest {
"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 class ConsumerGroupTest {
"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())
)
);

View File

@ -20,7 +20,6 @@ import org.apache.kafka.common.Uuid;
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 class ShareGroupBuilder {
return this;
}
public List<CoordinatorRecord> build(TopicsImage topicsImage) {
public List<CoordinatorRecord> build() {
List<CoordinatorRecord> records = new ArrayList<>();
// Add subscription records for members.

View File

@ -18,7 +18,8 @@ package org.apache.kafka.coordinator.group.modern.share;
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 class ShareGroupMemberTest {
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)

View File

@ -25,8 +25,9 @@ import org.apache.kafka.common.protocol.ApiKeys;
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 class ShareGroupTest {
new ShareGroupDescribeResponseData.Member().setMemberId("member2")
));
ShareGroupDescribeResponseData.DescribedGroup actual = shareGroup.asDescribedGroup(1, "assignorName",
new MetadataImageBuilder().build().topics());
new KRaftCoordinatorMetadataImage(new MetadataImageBuilder().build()));
assertEquals(expected, actual);
}

View File

@ -30,7 +30,8 @@ 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.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 class StreamsGroupTest {
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 class StreamsGroupTest {
.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"));

View File

@ -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.MemberAssignment;
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 class TargetAssignmentBuilderTest {
}
});
MetadataImage metadataImage = topicsImageBuilder.build();
CoordinatorMetadataImage metadataImage = new KRaftCoordinatorMetadataImage(topicsImageBuilder.build());
// Prepare the expected topology metadata.
TopologyMetadata topologyMetadata = new TopologyMetadata(metadataImage, subtopologies);

View File

@ -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 @@ import static org.mockito.Mockito.when;
class TopologyMetadataTest {
private MetadataImage metadataImage;
private CoordinatorMetadataImage metadataImage;
private SortedMap<String, ConfiguredSubtopology> 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);
}

View File

@ -19,7 +19,8 @@ package org.apache.kafka.coordinator.group.streams.topics;
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 @@ class EndpointToPartitionsManagerTest {
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 @@ class EndpointToPartitionsManagerTest {
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());

View File

@ -22,7 +22,8 @@ import org.apache.kafka.common.message.CreateTopicsRequestData.CreatableTopicCon
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 @@ class InternalTopicManagerTest {
// 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 @@ class InternalTopicManagerTest {
.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<String, CreatableTopic> internalTopicsToBeCreated = configuredTopology.internalTopicsToBeCreated();
assertEquals(2, internalTopicsToBeCreated.size());

View File

@ -19,6 +19,8 @@ package org.apache.kafka.jmh.assignor;
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 class AssignorBenchmarkUtils {
* @return A TopicsImage containing the topic ids, names and partition counts from the
* subscription metadata.
*/
public static MetadataImage createMetadataImage(
public static CoordinatorMetadataImage createMetadataImage(
List<String> allTopicNames,
int partitionsPerTopic
) {
@ -109,7 +111,7 @@ public class AssignorBenchmarkUtils {
);
}
return delta.apply(MetadataProvenance.EMPTY);
return new KRaftCoordinatorMetadataImage(delta.apply(MetadataProvenance.EMPTY));
}
/**

View File

@ -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.MemberSubscriptionAndAssignment
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 class ServerSideAssignorBenchmark {
private List<String> allTopicNames = List.of();
private MetadataImage metadataImage = MetadataImage.EMPTY;
private CoordinatorMetadataImage metadataImage = CoordinatorMetadataImage.EMPTY;
private TopicIds.TopicResolver topicResolver;
@ -148,7 +148,7 @@ public class ServerSideAssignorBenchmark {
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);
}

View File

@ -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.MemberSubscriptionAndAssignment
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 class ShareGroupAssignorBenchmark {
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);
}

View File

@ -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.StickyTaskAssignor;
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 class StreamsStickyAssignorBenchmark {
SortedMap<String, ConfiguredSubtopology> subtopologyMap = StreamsAssignorBenchmarkUtils.createSubtopologyMap(partitionCount, allTopicNames);
MetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
CoordinatorMetadataImage metadataImage = AssignorBenchmarkUtils.createMetadataImage(allTopicNames, partitionCount);
topologyDescriber = new TopologyMetadata(metadataImage, subtopologyMap);

View File

@ -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.SubscribedTopicDescriberImpl;
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<String> allTopicNames = List.of();
private MetadataImage metadataImage;
private CoordinatorMetadataImage metadataImage;
private TopicIds.TopicResolver topicResolver;
@ -127,7 +127,7 @@ public class TargetAssignmentBuilderBenchmark {
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);
}

View File

@ -20,6 +20,7 @@ import org.apache.kafka.common.Uuid;
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 class RegexResolutionBenchmark {
GROUP_ID,
LOG,
TIME,
image,
new KRaftCoordinatorMetadataImage(image),
Optional.empty(),
regexes
);

View File

@ -893,5 +893,8 @@ public class TopicsImageTest {
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);
}
}

View File

@ -30,8 +30,9 @@ import org.apache.kafka.common.message.WriteShareGroupStateRequestData;
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
);
}

View File

@ -45,6 +45,8 @@ import org.apache.kafka.common.utils.Time;
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.CoordinatorShardBuilderSuppli
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 class ShareCoordinatorService implements ShareCoordinator {
}
@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 @@ public class ShareCoordinatorService implements ShareCoordinator {
}
}
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();
}

View File

@ -41,6 +41,8 @@ import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
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.ShareUpdateKey;
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<CoordinatorRecord
private final TimelineHashMap<SharePartitionKey, Integer> leaderEpochMap;
private final TimelineHashMap<SharePartitionKey, Integer> snapshotUpdateCount;
private final TimelineHashMap<SharePartitionKey, Integer> stateEpochMap;
private MetadataImage metadataImage;
private CoordinatorMetadataImage metadataImage;
private final ShareCoordinatorOffsetsManager offsetsManager;
private final Time time;
@ -206,13 +206,13 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
}
@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 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
log.error("Metadata image is null");
return Optional.of(getWriteErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
if (metadataImage.topics().getTopic(topicId) == null ||
metadataImage.topics().getPartition(topicId, partitionId) == null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
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<CoordinatorMetadataImage.TopicMetadata> 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 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
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<CoordinatorMetadataImage.TopicMetadata> 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 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
return Optional.of(getDeleteErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
if (metadataImage.topics().getTopic(topicId) == null ||
metadataImage.topics().getPartition(topicId, partitionId) == null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
return Optional.of(getInitializeErrorCoordinatorResult(Errors.UNKNOWN_TOPIC_OR_PARTITION, null, topicId, partitionId));
}
if (metadataImage.topics().getTopic(topicId) == null ||
metadataImage.topics().getPartition(topicId, partitionId) == null) {
Optional<CoordinatorMetadataImage.TopicMetadata> 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));
}

View File

@ -41,12 +41,13 @@ import org.apache.kafka.common.utils.BufferSupplier;
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
)).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 @@ class ShareCoordinatorServiceTest {
));
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 @@ class ShareCoordinatorServiceTest {
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 @@ class ShareCoordinatorServiceTest {
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 @@ class ShareCoordinatorServiceTest {
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.

View File

@ -38,19 +38,21 @@ import org.apache.kafka.common.requests.WriteShareGroupStateResponse;
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.assertNotNull;
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 @@ class ShareCoordinatorShardTest {
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<String, String> 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 @@ class ShareCoordinatorShardTest {
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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
// 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 @@ class ShareCoordinatorShardTest {
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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
));
// 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 @@ class ShareCoordinatorShardTest {
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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
@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 @@ class ShareCoordinatorShardTest {
@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;