KAFKA-19338: Error on read/write of uninitialized share part. (#19861)

- Currently, read and write share state requests were allowed on
uninitialized share partitions (share partitions on which
initializeState has NOT been called). This should not be the case.
- This PR addresses the concern by adding error checks on read and
write. Other requests are allowed (initialize, readSummary, alter).
- Refactored `ShareCoordinatorShardTest` to reduce redundancy and added
some new tests.
- Some request/response classes have also been reformatted.

Reviewers: Andrew Schofield <aschofield@confluent.io>
This commit is contained in:
Sushant Mahajan 2025-06-03 15:56:38 +05:30 committed by GitHub
parent c5a78b0186
commit df93571f50
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
9 changed files with 400 additions and 299 deletions

View File

@ -59,15 +59,15 @@ public class DeleteShareGroupStateRequest extends AbstractRequest {
public DeleteShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) { public DeleteShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) {
List<DeleteShareGroupStateResponseData.DeleteStateResult> results = new ArrayList<>(); List<DeleteShareGroupStateResponseData.DeleteStateResult> results = new ArrayList<>();
data.topics().forEach( data.topics().forEach(
topicResult -> results.add(new DeleteShareGroupStateResponseData.DeleteStateResult() topicResult -> results.add(new DeleteShareGroupStateResponseData.DeleteStateResult()
.setTopicId(topicResult.topicId()) .setTopicId(topicResult.topicId())
.setPartitions(topicResult.partitions().stream() .setPartitions(topicResult.partitions().stream()
.map(partitionData -> new DeleteShareGroupStateResponseData.PartitionResult() .map(partitionData -> new DeleteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionData.partition()) .setPartition(partitionData.partition())
.setErrorCode(Errors.forException(e).code())) .setErrorCode(Errors.forException(e).code()))
.collect(Collectors.toList())))); .collect(Collectors.toList()))));
return new DeleteShareGroupStateResponse(new DeleteShareGroupStateResponseData() return new DeleteShareGroupStateResponse(new DeleteShareGroupStateResponseData()
.setResults(results)); .setResults(results));
} }
@Override @Override
@ -77,8 +77,8 @@ public class DeleteShareGroupStateRequest extends AbstractRequest {
public static DeleteShareGroupStateRequest parse(Readable readable, short version) { public static DeleteShareGroupStateRequest parse(Readable readable, short version) {
return new DeleteShareGroupStateRequest( return new DeleteShareGroupStateRequest(
new DeleteShareGroupStateRequestData(readable, version), new DeleteShareGroupStateRequestData(readable, version),
version version
); );
} }
} }

View File

@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Readable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -104,7 +103,7 @@ public class DeleteShareGroupStateResponse extends AbstractResponse {
public static DeleteShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) { public static DeleteShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) {
return new DeleteShareGroupStateResponseData().setResults( return new DeleteShareGroupStateResponseData().setResults(
Collections.singletonList(new DeleteShareGroupStateResponseData.DeleteStateResult() List.of(new DeleteShareGroupStateResponseData.DeleteStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult() .setPartitions(List.of(new DeleteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId) .setPartition(partitionId)

View File

@ -59,16 +59,16 @@ public class ReadShareGroupStateRequest extends AbstractRequest {
public ReadShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) { public ReadShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) {
List<ReadShareGroupStateResponseData.ReadStateResult> results = new ArrayList<>(); List<ReadShareGroupStateResponseData.ReadStateResult> results = new ArrayList<>();
data.topics().forEach( data.topics().forEach(
topicResult -> results.add(new ReadShareGroupStateResponseData.ReadStateResult() topicResult -> results.add(new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicResult.topicId()) .setTopicId(topicResult.topicId())
.setPartitions(topicResult.partitions().stream() .setPartitions(topicResult.partitions().stream()
.map(partitionData -> new ReadShareGroupStateResponseData.PartitionResult() .map(partitionData -> new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partitionData.partition()) .setPartition(partitionData.partition())
.setErrorCode(Errors.forException(e).code()) .setErrorCode(Errors.forException(e).code())
.setErrorMessage(Errors.forException(e).message())) .setErrorMessage(Errors.forException(e).message()))
.collect(Collectors.toList())))); .collect(Collectors.toList()))));
return new ReadShareGroupStateResponse(new ReadShareGroupStateResponseData() return new ReadShareGroupStateResponse(new ReadShareGroupStateResponseData()
.setResults(results)); .setResults(results));
} }
@Override @Override
@ -78,8 +78,8 @@ public class ReadShareGroupStateRequest extends AbstractRequest {
public static ReadShareGroupStateRequest parse(Readable readable, short version) { public static ReadShareGroupStateRequest parse(Readable readable, short version) {
return new ReadShareGroupStateRequest( return new ReadShareGroupStateRequest(
new ReadShareGroupStateRequestData(readable, version), new ReadShareGroupStateRequestData(readable, version),
version version
); );
} }
} }

View File

@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Readable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.EnumMap; import java.util.EnumMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -47,9 +46,9 @@ public class ReadShareGroupStateResponse extends AbstractResponse {
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> counts = new EnumMap<>(Errors.class); Map<Errors, Integer> counts = new EnumMap<>(Errors.class);
data.results().forEach( data.results().forEach(
result -> result.partitions().forEach( result -> result.partitions().forEach(
partitionResult -> updateErrorCounts(counts, Errors.forCode(partitionResult.errorCode())) partitionResult -> updateErrorCounts(counts, Errors.forCode(partitionResult.errorCode()))
) )
); );
return counts; return counts;
} }
@ -66,52 +65,52 @@ public class ReadShareGroupStateResponse extends AbstractResponse {
public static ReadShareGroupStateResponse parse(Readable readable, short version) { public static ReadShareGroupStateResponse parse(Readable readable, short version) {
return new ReadShareGroupStateResponse( return new ReadShareGroupStateResponse(
new ReadShareGroupStateResponseData(readable, version) new ReadShareGroupStateResponseData(readable, version)
); );
} }
public static ReadShareGroupStateResponseData toResponseData( public static ReadShareGroupStateResponseData toResponseData(
Uuid topicId, Uuid topicId,
int partition, int partition,
long startOffset, long startOffset,
int stateEpoch, int stateEpoch,
List<ReadShareGroupStateResponseData.StateBatch> stateBatches List<ReadShareGroupStateResponseData.StateBatch> stateBatches
) { ) {
return new ReadShareGroupStateResponseData() return new ReadShareGroupStateResponseData()
.setResults(Collections.singletonList( .setResults(List.of(
new ReadShareGroupStateResponseData.ReadStateResult() new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(Collections.singletonList( .setPartitions(List.of(
new ReadShareGroupStateResponseData.PartitionResult() new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partition) .setPartition(partition)
.setStartOffset(startOffset) .setStartOffset(startOffset)
.setStateEpoch(stateEpoch) .setStateEpoch(stateEpoch)
.setStateBatches(stateBatches) .setStateBatches(stateBatches)
)) ))
)); ));
} }
public static ReadShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) { public static ReadShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) {
return new ReadShareGroupStateResponseData().setResults( return new ReadShareGroupStateResponseData().setResults(
Collections.singletonList(new ReadShareGroupStateResponseData.ReadStateResult() List.of(new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(Collections.singletonList(new ReadShareGroupStateResponseData.PartitionResult() .setPartitions(List.of(new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId) .setPartition(partitionId)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setErrorMessage(errorMessage))))); .setErrorMessage(errorMessage)))));
} }
public static ReadShareGroupStateResponseData.PartitionResult toErrorResponsePartitionResult(int partitionId, Errors error, String errorMessage) { public static ReadShareGroupStateResponseData.PartitionResult toErrorResponsePartitionResult(int partitionId, Errors error, String errorMessage) {
return new ReadShareGroupStateResponseData.PartitionResult() return new ReadShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId) .setPartition(partitionId)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setErrorMessage(errorMessage); .setErrorMessage(errorMessage);
} }
public static ReadShareGroupStateResponseData.ReadStateResult toResponseReadStateResult(Uuid topicId, List<ReadShareGroupStateResponseData.PartitionResult> partitionResults) { public static ReadShareGroupStateResponseData.ReadStateResult toResponseReadStateResult(Uuid topicId, List<ReadShareGroupStateResponseData.PartitionResult> partitionResults) {
return new ReadShareGroupStateResponseData.ReadStateResult() return new ReadShareGroupStateResponseData.ReadStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(partitionResults); .setPartitions(partitionResults);
} }
public static ReadShareGroupStateResponseData toGlobalErrorResponse(ReadShareGroupStateRequestData request, Errors error) { public static ReadShareGroupStateResponseData toGlobalErrorResponse(ReadShareGroupStateRequestData request, Errors error) {

View File

@ -59,16 +59,16 @@ public class WriteShareGroupStateRequest extends AbstractRequest {
public WriteShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) { public WriteShareGroupStateResponse getErrorResponse(int throttleTimeMs, Throwable e) {
List<WriteShareGroupStateResponseData.WriteStateResult> results = new ArrayList<>(); List<WriteShareGroupStateResponseData.WriteStateResult> results = new ArrayList<>();
data.topics().forEach( data.topics().forEach(
topicResult -> results.add(new WriteShareGroupStateResponseData.WriteStateResult() topicResult -> results.add(new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicResult.topicId()) .setTopicId(topicResult.topicId())
.setPartitions(topicResult.partitions().stream() .setPartitions(topicResult.partitions().stream()
.map(partitionData -> new WriteShareGroupStateResponseData.PartitionResult() .map(partitionData -> new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionData.partition()) .setPartition(partitionData.partition())
.setErrorCode(Errors.forException(e).code()) .setErrorCode(Errors.forException(e).code())
.setErrorMessage(Errors.forException(e).message())) .setErrorMessage(Errors.forException(e).message()))
.collect(Collectors.toList())))); .collect(Collectors.toList()))));
return new WriteShareGroupStateResponse(new WriteShareGroupStateResponseData() return new WriteShareGroupStateResponse(new WriteShareGroupStateResponseData()
.setResults(results)); .setResults(results));
} }
@Override @Override
@ -78,8 +78,8 @@ public class WriteShareGroupStateRequest extends AbstractRequest {
public static WriteShareGroupStateRequest parse(Readable readable, short version) { public static WriteShareGroupStateRequest parse(Readable readable, short version) {
return new WriteShareGroupStateRequest( return new WriteShareGroupStateRequest(
new WriteShareGroupStateRequestData(readable, version), new WriteShareGroupStateRequestData(readable, version),
version version
); );
} }
} }

View File

@ -25,7 +25,6 @@ import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.protocol.Readable; import org.apache.kafka.common.protocol.Readable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -47,9 +46,9 @@ public class WriteShareGroupStateResponse extends AbstractResponse {
public Map<Errors, Integer> errorCounts() { public Map<Errors, Integer> errorCounts() {
Map<Errors, Integer> counts = new HashMap<>(); Map<Errors, Integer> counts = new HashMap<>();
data.results().forEach( data.results().forEach(
result -> result.partitions().forEach( result -> result.partitions().forEach(
partitionResult -> updateErrorCounts(counts, Errors.forCode(partitionResult.errorCode())) partitionResult -> updateErrorCounts(counts, Errors.forCode(partitionResult.errorCode()))
) )
); );
return counts; return counts;
} }
@ -66,47 +65,47 @@ public class WriteShareGroupStateResponse extends AbstractResponse {
public static WriteShareGroupStateResponse parse(Readable readable, short version) { public static WriteShareGroupStateResponse parse(Readable readable, short version) {
return new WriteShareGroupStateResponse( return new WriteShareGroupStateResponse(
new WriteShareGroupStateResponseData(readable, version) new WriteShareGroupStateResponseData(readable, version)
); );
} }
public static WriteShareGroupStateResponseData toResponseData(Uuid topicId, int partitionId) { public static WriteShareGroupStateResponseData toResponseData(Uuid topicId, int partitionId) {
return new WriteShareGroupStateResponseData() return new WriteShareGroupStateResponseData()
.setResults(Collections.singletonList( .setResults(List.of(
new WriteShareGroupStateResponseData.WriteStateResult() new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(Collections.singletonList( .setPartitions(List.of(
new WriteShareGroupStateResponseData.PartitionResult() new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId))))); .setPartition(partitionId)))));
} }
public static WriteShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) { public static WriteShareGroupStateResponseData toErrorResponseData(Uuid topicId, int partitionId, Errors error, String errorMessage) {
WriteShareGroupStateResponseData responseData = new WriteShareGroupStateResponseData(); WriteShareGroupStateResponseData responseData = new WriteShareGroupStateResponseData();
responseData.setResults(Collections.singletonList(new WriteShareGroupStateResponseData.WriteStateResult() responseData.setResults(List.of(new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(Collections.singletonList(new WriteShareGroupStateResponseData.PartitionResult() .setPartitions(List.of(new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId) .setPartition(partitionId)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setErrorMessage(errorMessage))))); .setErrorMessage(errorMessage)))));
return responseData; return responseData;
} }
public static WriteShareGroupStateResponseData.PartitionResult toErrorResponsePartitionResult(int partitionId, Errors error, String errorMessage) { public static WriteShareGroupStateResponseData.PartitionResult toErrorResponsePartitionResult(int partitionId, Errors error, String errorMessage) {
return new WriteShareGroupStateResponseData.PartitionResult() return new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId) .setPartition(partitionId)
.setErrorCode(error.code()) .setErrorCode(error.code())
.setErrorMessage(errorMessage); .setErrorMessage(errorMessage);
} }
public static WriteShareGroupStateResponseData.WriteStateResult toResponseWriteStateResult(Uuid topicId, List<WriteShareGroupStateResponseData.PartitionResult> partitionResults) { public static WriteShareGroupStateResponseData.WriteStateResult toResponseWriteStateResult(Uuid topicId, List<WriteShareGroupStateResponseData.PartitionResult> partitionResults) {
return new WriteShareGroupStateResponseData.WriteStateResult() return new WriteShareGroupStateResponseData.WriteStateResult()
.setTopicId(topicId) .setTopicId(topicId)
.setPartitions(partitionResults); .setPartitions(partitionResults);
} }
public static WriteShareGroupStateResponseData.PartitionResult toResponsePartitionResult(int partitionId) { public static WriteShareGroupStateResponseData.PartitionResult toResponsePartitionResult(int partitionId) {
return new WriteShareGroupStateResponseData.PartitionResult() return new WriteShareGroupStateResponseData.PartitionResult()
.setPartition(partitionId); .setPartition(partitionId);
} }
public static WriteShareGroupStateResponseData toGlobalErrorResponse(WriteShareGroupStateRequestData request, Errors error) { public static WriteShareGroupStateResponseData toGlobalErrorResponse(WriteShareGroupStateRequestData request, Errors error) {

View File

@ -17,12 +17,13 @@
package kafka.server package kafka.server
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.kafka.clients.admin.DescribeShareGroupsOptions
import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type} import org.apache.kafka.common.test.api.{ClusterConfigProperty, ClusterFeature, ClusterTest, ClusterTestDefaults, ClusterTests, Type}
import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords import org.apache.kafka.common.message.ShareFetchResponseData.AcquiredRecords
import org.apache.kafka.common.message.{ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData} import org.apache.kafka.common.message.{FindCoordinatorRequestData, ShareAcknowledgeRequestData, ShareAcknowledgeResponseData, ShareFetchRequestData, ShareFetchResponseData, ShareGroupHeartbeatRequestData}
import org.apache.kafka.common.protocol.Errors import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid} import org.apache.kafka.common.{TopicIdPartition, TopicPartition, Uuid}
import org.apache.kafka.common.requests.{ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareRequestMetadata} import org.apache.kafka.common.requests.{FindCoordinatorRequest, FindCoordinatorResponse, ShareAcknowledgeRequest, ShareAcknowledgeResponse, ShareFetchRequest, ShareFetchResponse, ShareGroupHeartbeatRequest, ShareGroupHeartbeatResponse, ShareRequestMetadata}
import org.apache.kafka.common.test.ClusterInstance import org.apache.kafka.common.test.ClusterInstance
import org.apache.kafka.server.common.Feature import org.apache.kafka.server.common.Feature
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
@ -109,7 +110,8 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
) )
def testShareFetchRequestToNonLeaderReplica(): Unit = { def testShareFetchRequestToNonLeaderReplica(): Unit = {
val groupId: String = "group" val groupId: String = "group"
val metadata: ShareRequestMetadata = new ShareRequestMetadata(Uuid.randomUuid(), ShareRequestMetadata.INITIAL_EPOCH) val memberId: Uuid = Uuid.randomUuid()
val metadata: ShareRequestMetadata = new ShareRequestMetadata(memberId, ShareRequestMetadata.INITIAL_EPOCH)
val topic = "topic" val topic = "topic"
val partition = 0 val partition = 0
@ -129,6 +131,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connect(nonReplicaId) val socket: Socket = connect(nonReplicaId)
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 1))
// Send the share fetch request to the non-replica and verify the error code // Send the share fetch request to the non-replica and verify the error code
val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty) val shareFetchRequest = createShareFetchRequest(groupId, metadata, send, Seq.empty, Map.empty)
val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket) val shareFetchResponse = IntegrationTestUtils.sendAndReceive[ShareFetchResponse](shareFetchRequest, socket)
@ -172,6 +177,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -238,6 +246,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partitions // Send the first share fetch request to initialize the share partitions
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -349,6 +360,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket2: Socket = connect(leader2) val socket2: Socket = connect(leader2)
val socket3: Socket = connect(leader3) val socket3: Socket = connect(leader3)
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partitions // Send the first share fetch request to initialize the share partitions
// Create different share fetch requests for different partitions as they may have leaders on separate brokers // Create different share fetch requests for different partitions as they may have leaders on separate brokers
var shareFetchRequest1 = createShareFetchRequest(groupId, metadata, send1, Seq.empty, acknowledgementsMap) var shareFetchRequest1 = createShareFetchRequest(groupId, metadata, send1, Seq.empty, acknowledgementsMap)
@ -456,6 +470,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize share partitions // Send the first share fetch request to initialize share partitions
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -573,6 +590,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket, 15000) sendFirstShareFetchRequest(memberId, groupId, send, socket, 15000)
@ -693,6 +713,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partiion // Send the first share fetch request to initialize the share partiion
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -805,6 +828,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -921,6 +947,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1036,6 +1065,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1158,6 +1190,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the shar partition // Send the first share fetch request to initialize the shar partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1323,6 +1358,11 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket2: Socket = connectAny() val socket2: Socket = connectAny()
val socket3: Socket = connectAny() val socket3: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId1, groupId, Map[String, Int](topic -> 3))
shareHeartbeat(memberId2, groupId, Map[String, Int](topic -> 3))
shareHeartbeat(memberId3, groupId, Map[String, Int](topic -> 3))
// Sending a dummy share fetch request to initialize the share partition // Sending a dummy share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId1, groupId, send, socket1) sendFirstShareFetchRequest(memberId1, groupId, send, socket1)
@ -1420,6 +1460,11 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket2: Socket = connectAny() val socket2: Socket = connectAny()
val socket3: Socket = connectAny() val socket3: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId1, groupId1, Map[String, Int](topic -> 3))
shareHeartbeat(memberId2, groupId2, Map[String, Int](topic -> 3))
shareHeartbeat(memberId3, groupId3, Map[String, Int](topic -> 3))
// Sending 3 dummy share Fetch Requests with to inititlaize the share partitions for each share group\ // Sending 3 dummy share Fetch Requests with to inititlaize the share partitions for each share group\
sendFirstShareFetchRequest(memberId1, groupId1, send, socket1) sendFirstShareFetchRequest(memberId1, groupId1, send, socket1)
sendFirstShareFetchRequest(memberId2, groupId2, send, socket2) sendFirstShareFetchRequest(memberId2, groupId2, send, socket2)
@ -1513,6 +1558,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1623,6 +1671,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1844,6 +1895,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1918,6 +1972,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -1998,6 +2055,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -2161,6 +2221,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -2242,6 +2305,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -2339,6 +2405,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -2403,6 +2472,9 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
val socket: Socket = connectAny() val socket: Socket = connectAny()
createOffsetsTopic()
shareHeartbeat(memberId, groupId, Map[String, Int](topic -> 3))
// Send the first share fetch request to initialize the share partition // Send the first share fetch request to initialize the share partition
sendFirstShareFetchRequest(memberId, groupId, send, socket) sendFirstShareFetchRequest(memberId, groupId, send, socket)
@ -2455,6 +2527,40 @@ class ShareFetchAcknowledgeRequestTest(cluster: ClusterInstance) extends GroupCo
}, "Share fetch request failed", 5000) }, "Share fetch request failed", 5000)
} }
private def shareHeartbeat(memberId: Uuid, groupId: String, topics: Map[String, Int]): Unit = {
val coordResp = connectAndReceive[FindCoordinatorResponse](new FindCoordinatorRequest.Builder(new FindCoordinatorRequestData()
.setKey(groupId)
.setKeyType(0.toByte)
).build(0)
)
val shareGroupHeartbeatRequest = new ShareGroupHeartbeatRequest.Builder(
new ShareGroupHeartbeatRequestData()
.setMemberId(memberId.toString)
.setGroupId(groupId)
.setMemberEpoch(0)
.setSubscribedTopicNames(topics.keys.toList.asJava)
).build()
TestUtils.waitUntilTrue(() => {
val resp = connectAndReceive[ShareGroupHeartbeatResponse](shareGroupHeartbeatRequest, coordResp.node().id())
resp.data().errorCode() == Errors.NONE.code() && assignment(memberId.toString, groupId)
}, "Heartbeat failed")
}
private def assignment(memberId: String, groupId: String): Boolean = {
val admin = cluster.admin()
val isAssigned = admin.describeShareGroups(List(groupId).asJava, new DescribeShareGroupsOptions().includeAuthorizedOperations(true))
.describedGroups()
.get(groupId)
.get()
.members()
.asScala.count(desc => desc.consumerId() == memberId && !desc.assignment().topicPartitions().isEmpty) > 0
admin.close()
isAssigned
}
private def expectedAcquiredRecords(firstOffsets: util.List[Long], lastOffsets: util.List[Long], deliveryCounts: util.List[Int]): util.List[AcquiredRecords] = { private def expectedAcquiredRecords(firstOffsets: util.List[Long], lastOffsets: util.List[Long], deliveryCounts: util.List[Int]): util.List[AcquiredRecords] = {
val acquiredRecordsList: util.List[AcquiredRecords] = new util.ArrayList() val acquiredRecordsList: util.List[AcquiredRecords] = new util.ArrayList()
for (i <- firstOffsets.asScala.indices) { for (i <- firstOffsets.asScala.indices) {

View File

@ -89,6 +89,8 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
public static final Exception NULL_TOPIC_ID = new Exception("The topic id cannot be null."); public static final Exception NULL_TOPIC_ID = new Exception("The topic id cannot be null.");
public static final Exception NEGATIVE_PARTITION_ID = new Exception("The partition id cannot be a negative number."); public static final Exception NEGATIVE_PARTITION_ID = new Exception("The partition id cannot be a negative number.");
public static final Exception WRITE_UNINITIALIZED_SHARE_PARTITION = new Exception("Write operation on uninitialized share partition not allowed.");
public static final Exception READ_UNINITIALIZED_SHARE_PARTITION = new Exception("Read operation on uninitialized share partition not allowed.");
public static class Builder implements CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> { public static class Builder implements CoordinatorShardBuilder<ShareCoordinatorShard, CoordinatorRecord> {
private final ShareCoordinatorConfig config; private final ShareCoordinatorConfig config;
@ -363,38 +365,24 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
int leaderEpoch = partitionData.leaderEpoch(); int leaderEpoch = partitionData.leaderEpoch();
SharePartitionKey key = SharePartitionKey.getInstance(request.groupId(), topicId, partitionId); SharePartitionKey key = SharePartitionKey.getInstance(request.groupId(), topicId, partitionId);
ReadShareGroupStateResponseData responseData = null; ShareGroupOffset offsetValue = shareStateMap.get(key);
List<ReadShareGroupStateResponseData.StateBatch> stateBatches = (offsetValue.stateBatches() != null && !offsetValue.stateBatches().isEmpty()) ?
offsetValue.stateBatches().stream()
.map(
stateBatch -> new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(stateBatch.firstOffset())
.setLastOffset(stateBatch.lastOffset())
.setDeliveryState(stateBatch.deliveryState())
.setDeliveryCount(stateBatch.deliveryCount())
).toList() : List.of();
if (!shareStateMap.containsKey(key)) { ReadShareGroupStateResponseData responseData = ReadShareGroupStateResponse.toResponseData(
// Leader epoch update might be needed topicId,
responseData = ReadShareGroupStateResponse.toResponseData( partitionId,
topicId, offsetValue.startOffset(),
partitionId, offsetValue.stateEpoch(),
PartitionFactory.UNINITIALIZED_START_OFFSET, stateBatches
PartitionFactory.DEFAULT_STATE_EPOCH, );
List.of()
);
} else {
// Leader epoch update might be needed
ShareGroupOffset offsetValue = shareStateMap.get(key);
List<ReadShareGroupStateResponseData.StateBatch> stateBatches = (offsetValue.stateBatches() != null && !offsetValue.stateBatches().isEmpty()) ?
offsetValue.stateBatches().stream()
.map(
stateBatch -> new ReadShareGroupStateResponseData.StateBatch()
.setFirstOffset(stateBatch.firstOffset())
.setLastOffset(stateBatch.lastOffset())
.setDeliveryState(stateBatch.deliveryState())
.setDeliveryCount(stateBatch.deliveryCount())
).toList() : List.of();
responseData = ReadShareGroupStateResponse.toResponseData(
topicId,
partitionId,
offsetValue.startOffset(),
offsetValue.stateEpoch(),
stateBatches
);
}
// Optimization in case leaderEpoch update is not required. // Optimization in case leaderEpoch update is not required.
if (leaderEpoch == -1 || if (leaderEpoch == -1 ||
@ -644,9 +632,9 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
/** /**
* Util method to generate a ShareSnapshot or ShareUpdate type record for a key, based on various conditions. * Util method to generate a ShareSnapshot or ShareUpdate type record for a key, based on various conditions.
* <p> * <p>
* If no snapshot has been created for the key => create a new ShareSnapshot record * If number of ShareUpdate records for key >= max allowed per snapshot per key or stateEpoch is highest
* else if number of ShareUpdate records for key >= max allowed per snapshot per key => create a new ShareSnapshot record * seen so far => create a new ShareSnapshot record else create a new ShareUpdate record. This method assumes
* else create a new ShareUpdate record * that share partition key is present in shareStateMap since it should be called on initialized share partitions.
* *
* @param partitionData - Represents the data which should be written into the share state record. * @param partitionData - Represents the data which should be written into the share state record.
* @param key - The {@link SharePartitionKey} object. * @param key - The {@link SharePartitionKey} object.
@ -658,28 +646,14 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
) { ) {
long timestamp = time.milliseconds(); long timestamp = time.milliseconds();
int updatesPerSnapshotLimit = config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot(); int updatesPerSnapshotLimit = config.shareCoordinatorSnapshotUpdateRecordsPerSnapshot();
if (!shareStateMap.containsKey(key)) { if (snapshotUpdateCount.getOrDefault(key, 0) >= updatesPerSnapshotLimit || partitionData.stateEpoch() > shareStateMap.get(key).stateEpoch()) {
// Since this is the first time we are getting a write request for key, we should be creating a share snapshot record.
// The incoming partition data could have overlapping state batches, we must merge them
return ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
key.groupId(), key.topicId(), partitionData.partition(),
new ShareGroupOffset.Builder()
.setSnapshotEpoch(0)
.setStartOffset(partitionData.startOffset())
.setLeaderEpoch(partitionData.leaderEpoch())
.setStateEpoch(partitionData.stateEpoch())
.setStateBatches(mergeBatches(List.of(), partitionData))
.setCreateTimestamp(timestamp)
.setWriteTimestamp(timestamp)
.build());
} else if (snapshotUpdateCount.getOrDefault(key, 0) >= updatesPerSnapshotLimit || partitionData.stateEpoch() > shareStateMap.get(key).stateEpoch()) {
ShareGroupOffset currentState = shareStateMap.get(key); // shareStateMap will have the entry as containsKey is true ShareGroupOffset currentState = shareStateMap.get(key); // shareStateMap will have the entry as containsKey is true
int newLeaderEpoch = partitionData.leaderEpoch() == -1 ? currentState.leaderEpoch() : partitionData.leaderEpoch(); int newLeaderEpoch = partitionData.leaderEpoch() == -1 ? currentState.leaderEpoch() : partitionData.leaderEpoch();
int newStateEpoch = partitionData.stateEpoch() == -1 ? currentState.stateEpoch() : partitionData.stateEpoch(); int newStateEpoch = partitionData.stateEpoch() == -1 ? currentState.stateEpoch() : partitionData.stateEpoch();
long newStartOffset = partitionData.startOffset() == -1 ? currentState.startOffset() : partitionData.startOffset(); long newStartOffset = partitionData.startOffset() == -1 ? currentState.startOffset() : partitionData.startOffset();
// Since the number of update records for this share part key exceeds snapshotUpdateRecordsPerSnapshot, // Since the number of update records for this share part key exceeds snapshotUpdateRecordsPerSnapshot
// we should be creating a share snapshot record. // or state epoch has incremented, we should be creating a share snapshot record.
// The incoming partition data could have overlapping state batches, we must merge them. // The incoming partition data could have overlapping state batches, we must merge them.
return ShareCoordinatorRecordHelpers.newShareSnapshotRecord( return ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
key.groupId(), key.topicId(), partitionData.partition(), key.groupId(), key.topicId(), partitionData.partition(),
@ -772,6 +746,11 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
} }
SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId); SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId);
if (!shareStateMap.containsKey(mapKey)) {
return Optional.of(getWriteErrorCoordinatorResult(Errors.INVALID_REQUEST, WRITE_UNINITIALIZED_SHARE_PARTITION, topicId, partitionId));
}
if (partitionData.leaderEpoch() != -1 && leaderEpochMap.containsKey(mapKey) && leaderEpochMap.get(mapKey) > partitionData.leaderEpoch()) { if (partitionData.leaderEpoch() != -1 && leaderEpochMap.containsKey(mapKey) && leaderEpochMap.get(mapKey) > partitionData.leaderEpoch()) {
log.error("Write request leader epoch is smaller than last recorded current: {}, requested: {}.", leaderEpochMap.get(mapKey), partitionData.leaderEpoch()); log.error("Write request leader epoch is smaller than last recorded current: {}, requested: {}.", leaderEpochMap.get(mapKey), partitionData.leaderEpoch());
return Optional.of(getWriteErrorCoordinatorResult(Errors.FENCED_LEADER_EPOCH, null, topicId, partitionId)); return Optional.of(getWriteErrorCoordinatorResult(Errors.FENCED_LEADER_EPOCH, null, topicId, partitionId));
@ -814,6 +793,13 @@ public class ShareCoordinatorShard implements CoordinatorShard<CoordinatorRecord
} }
SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId); SharePartitionKey mapKey = SharePartitionKey.getInstance(groupId, topicId, partitionId);
if (!shareStateMap.containsKey(mapKey)) {
log.error("Read on uninitialized share partition {}", mapKey);
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(
topicId, partitionId, Errors.INVALID_REQUEST, READ_UNINITIALIZED_SHARE_PARTITION.getMessage()));
}
if (leaderEpochMap.containsKey(mapKey) && leaderEpochMap.get(mapKey) > partitionData.leaderEpoch()) { if (leaderEpochMap.containsKey(mapKey) && leaderEpochMap.get(mapKey) > partitionData.leaderEpoch()) {
log.error("Read request leader epoch is smaller than last recorded current: {}, requested: {}.", leaderEpochMap.get(mapKey), partitionData.leaderEpoch()); log.error("Read request leader epoch is smaller than last recorded current: {}, requested: {}.", leaderEpochMap.get(mapKey), partitionData.leaderEpoch());
return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.FENCED_LEADER_EPOCH, Errors.FENCED_LEADER_EPOCH.message())); return Optional.of(ReadShareGroupStateResponse.toErrorResponseData(topicId, partitionId, Errors.FENCED_LEADER_EPOCH, Errors.FENCED_LEADER_EPOCH.message()));

View File

@ -57,6 +57,7 @@ import org.apache.kafka.server.share.persister.PartitionFactory;
import org.apache.kafka.server.share.persister.PersisterStateBatch; import org.apache.kafka.server.share.persister.PersisterStateBatch;
import org.apache.kafka.timeline.SnapshotRegistry; import org.apache.kafka.timeline.SnapshotRegistry;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.util.HashMap; import java.util.HashMap;
@ -86,6 +87,7 @@ class ShareCoordinatorShardTest {
private static final Uuid TOPIC_ID = Uuid.randomUuid(); private static final Uuid TOPIC_ID = Uuid.randomUuid();
private static final Uuid TOPIC_ID_2 = Uuid.randomUuid(); private static final Uuid TOPIC_ID_2 = Uuid.randomUuid();
private static final int PARTITION = 0; private static final int PARTITION = 0;
private static final SharePartitionKey SHARE_PARTITION_KEY = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
private static final Time TIME = new MockTime(); private static final Time TIME = new MockTime();
public static class ShareCoordinatorShardBuilder { public static class ShareCoordinatorShardBuilder {
@ -141,7 +143,6 @@ class ShareCoordinatorShardTest {
} }
private void writeAndReplayRecord(ShareCoordinatorShard shard, int leaderEpoch) { private void writeAndReplayRecord(ShareCoordinatorShard shard, int leaderEpoch) {
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new WriteShareGroupStateRequestData.WriteStateData() .setTopics(List.of(new WriteShareGroupStateRequestData.WriteStateData()
@ -162,10 +163,15 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
} }
private ShareCoordinatorShard shard;
@BeforeEach
public void setUp() {
shard = new ShareCoordinatorShardBuilder().build();
}
@Test @Test
public void testReplayWithShareSnapshot() { public void testReplayWithShareSnapshot() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
long offset = 0; long offset = 0;
long producerId = 0; long producerId = 0;
short producerEpoch = 0; short producerEpoch = 0;
@ -230,10 +236,39 @@ class ShareCoordinatorShardTest {
} }
@Test @Test
public void testWriteStateSuccess() { public void testWriteFailsOnUninitializedPartition() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(List.of(new WriteShareGroupStateRequestData.WriteStateData()
.setTopicId(TOPIC_ID)
.setPartitions(List.of(new WriteShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setStartOffset(0)
.setStateEpoch(0)
.setLeaderEpoch(0)
.setStateBatches(List.of(new WriteShareGroupStateRequestData.StateBatch()
.setFirstOffset(0)
.setLastOffset(10)
.setDeliveryCount((short) 1)
.setDeliveryState((byte) 0)))))));
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); CoordinatorResult<WriteShareGroupStateResponseData, CoordinatorRecord> result = shard.writeState(request);
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toErrorResponseData(
TOPIC_ID, PARTITION,
Errors.INVALID_REQUEST,
ShareCoordinatorShard.WRITE_UNINITIALIZED_SHARE_PARTITION.getMessage()
);
List<CoordinatorRecord> expectedRecords = List.of();
assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records());
assertNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
}
@Test
public void testWriteStateSuccess() {
initSharePartition(shard, SHARE_PARTITION_KEY);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -255,17 +290,17 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds())
)); ));
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds())
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
} }
@ -276,9 +311,10 @@ class ShareCoordinatorShardTest {
// a higher state epoch in a request forces snapshot creation, even if number of share updates // a higher state epoch in a request forces snapshot creation, even if number of share updates
// have not breached the updates/snapshots limit. // have not breached the updates/snapshots limit.
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); int stateEpoch = 1;
int stateEpoch = 0; int snapshotEpoch = 0;
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
initSharePartition(shard, SHARE_PARTITION_KEY);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -299,19 +335,20 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
snapshotEpoch++; // Since state epoch increased.
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
)); ));
assertEquals(0, shard.getShareStateMapValue(shareCoordinatorKey).snapshotEpoch()); assertEquals(1, shard.getShareStateMapValue(SHARE_PARTITION_KEY).snapshotEpoch());
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
// State epoch stays same so share update. // State epoch stays same so share update.
@ -336,18 +373,18 @@ class ShareCoordinatorShardTest {
expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord( expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
)); ));
// Snapshot epoch did not increase // Snapshot epoch did not increase
assertEquals(0, shard.getShareStateMapValue(shareCoordinatorKey).snapshotEpoch()); assertEquals(1, shard.getShareStateMapValue(SHARE_PARTITION_KEY).snapshotEpoch());
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard(), times(2)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard(), times(2)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
// State epoch incremented so share snapshot. // State epoch incremented so share snapshot.
@ -370,28 +407,27 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
snapshotEpoch++; // Since state epoch increased
expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), 1, TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
)); ));
// Snapshot epoch increased. // Snapshot epoch increased.
assertEquals(1, shard.getShareStateMapValue(shareCoordinatorKey).snapshotEpoch()); assertEquals(2, shard.getShareStateMapValue(SHARE_PARTITION_KEY).snapshotEpoch());
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), 1, TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), snapshotEpoch, TIME.milliseconds())
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard(), times(3)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard(), times(3)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
} }
@Test @Test
public void testSubsequentWriteStateSnapshotEpochUpdatesSuccessfully() { public void testSubsequentWriteStateSnapshotEpochUpdatesSuccessfully() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -428,7 +464,7 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), TIME.milliseconds())
)); ));
@ -436,8 +472,8 @@ class ShareCoordinatorShardTest {
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()), assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey)); shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
result = shard.writeState(request2); result = shard.writeState(request2);
@ -454,24 +490,20 @@ class ShareCoordinatorShardTest {
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
ShareGroupOffset incrementalUpdate = groupOffset(expectedRecords.get(0).value().message()); ShareGroupOffset incrementalUpdate = groupOffset(expectedRecords.get(0).value().message());
ShareGroupOffset combinedState = shard.getShareStateMapValue(shareCoordinatorKey); ShareGroupOffset combinedState = shard.getShareStateMapValue(SHARE_PARTITION_KEY);
assertEquals(incrementalUpdate.snapshotEpoch(), combinedState.snapshotEpoch()); assertEquals(incrementalUpdate.snapshotEpoch(), combinedState.snapshotEpoch());
assertEquals(incrementalUpdate.leaderEpoch(), combinedState.leaderEpoch()); assertEquals(incrementalUpdate.leaderEpoch(), combinedState.leaderEpoch());
assertEquals(incrementalUpdate.startOffset(), combinedState.startOffset()); assertEquals(incrementalUpdate.startOffset(), combinedState.startOffset());
// The batches should have combined to 1 since same state. // The batches should have combined to 1 since same state.
assertEquals(List.of(new PersisterStateBatch(0, 20, (byte) 0, (short) 1)), assertEquals(List.of(new PersisterStateBatch(0, 20, (byte) 0, (short) 1)),
combinedState.stateBatches()); combinedState.stateBatches());
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testWriteStateInvalidRequestData() { public void testWriteStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
int partition = -1; int partition = -1;
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new WriteShareGroupStateRequestData.WriteStateData() .setTopics(List.of(new WriteShareGroupStateRequestData.WriteStateData()
@ -496,16 +528,15 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey)); assertNull(shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testWriteNullMetadataImage() { public void testWriteNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
shard.onNewMetadataImage(null, null);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); shard.onNewMetadataImage(null, null);
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -530,16 +561,12 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(-1, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getShareStateMapValue(shareCoordinatorKey));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey));
} }
@Test @Test
public void testWriteStateFencedLeaderEpochError() { public void testWriteStateFencedLeaderEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -576,7 +603,7 @@ class ShareCoordinatorShardTest {
shard.replay(0L, 0L, (short) 0, result.records().get(0)); shard.replay(0L, 0L, (short) 0, result.records().get(0));
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), TIME.milliseconds())
)); ));
@ -584,8 +611,8 @@ class ShareCoordinatorShardTest {
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()), assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey)); shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(5, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
result = shard.writeState(request2); result = shard.writeState(request2);
@ -598,14 +625,12 @@ class ShareCoordinatorShardTest {
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
// No changes to the leaderMap. // No changes to the leaderMap.
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(5, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testWriteStateFencedStateEpochError() { public void testWriteStateFencedStateEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request1 = new WriteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -643,15 +668,15 @@ class ShareCoordinatorShardTest {
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request1.topics().get(0).partitions().get(0), 1, TIME.milliseconds())
)); ));
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()), assertEquals(groupOffset(expectedRecords.get(0).value().message()),
shard.getShareStateMapValue(shareCoordinatorKey)); shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(5, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(5, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
result = shard.writeState(request2); result = shard.writeState(request2);
@ -664,15 +689,34 @@ class ShareCoordinatorShardTest {
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
// No changes to the stateEpochMap. // No changes to the stateEpochMap.
assertEquals(1, shard.getStateEpochMapValue(shareCoordinatorKey)); assertEquals(1, shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
}
@Test
public void testReadFailsOnUninitializedPartition() {
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID)
.setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData()
.setTopicId(TOPIC_ID)
.setPartitions(List.of(new ReadShareGroupStateRequestData.PartitionData()
.setPartition(PARTITION)
.setLeaderEpoch(1)))));
CoordinatorResult<ReadShareGroupStateResponseData, CoordinatorRecord> result = shard.readStateAndMaybeUpdateLeaderEpoch(request);
assertEquals(ReadShareGroupStateResponse.toErrorResponseData(
TOPIC_ID,
PARTITION,
Errors.INVALID_REQUEST,
ShareCoordinatorShard.READ_UNINITIALIZED_SHARE_PARTITION.getMessage()
), result.response());
assertNull(shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadStateSuccess() { public void testReadStateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey coordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
writeAndReplayDefaultRecord(shard); writeAndReplayDefaultRecord(shard);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
@ -698,15 +742,12 @@ class ShareCoordinatorShardTest {
) )
), result.response()); ), result.response());
assertEquals(0, shard.getLeaderMapValue(coordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadStateSummarySuccess() { public void testReadStateSummarySuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey coordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
writeAndReplayDefaultRecord(shard); writeAndReplayDefaultRecord(shard);
ReadShareGroupStateSummaryRequestData request = new ReadShareGroupStateSummaryRequestData() ReadShareGroupStateSummaryRequestData request = new ReadShareGroupStateSummaryRequestData()
@ -727,19 +768,15 @@ class ShareCoordinatorShardTest {
0 0
), result.response()); ), result.response());
assertEquals(0, shard.getLeaderMapValue(coordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadStateInvalidRequestData() { public void testReadStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
int partition = -1;
writeAndReplayDefaultRecord(shard); writeAndReplayDefaultRecord(shard);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); int partition = -1;
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData() .setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData()
@ -756,19 +793,15 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
// Leader epoch should not be changed because the request failed. // Leader epoch should not be changed because the request failed.
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadStateSummaryInvalidRequestData() { public void testReadStateSummaryInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
int partition = -1;
writeAndReplayDefaultRecord(shard); writeAndReplayDefaultRecord(shard);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); int partition = -1;
ReadShareGroupStateSummaryRequestData request = new ReadShareGroupStateSummaryRequestData() ReadShareGroupStateSummaryRequestData request = new ReadShareGroupStateSummaryRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData() .setTopics(List.of(new ReadShareGroupStateSummaryRequestData.ReadStateSummaryData()
@ -785,19 +818,16 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
// Leader epoch should not be changed because the request failed. // Leader epoch should not be changed because the request failed.
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadNullMetadataImage() { public void testReadNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
writeAndReplayDefaultRecord(shard); writeAndReplayDefaultRecord(shard);
shard.onNewMetadataImage(null, null); shard.onNewMetadataImage(null, null);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData() .setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData()
@ -814,19 +844,16 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
// Leader epoch should not be changed because the request failed. // Leader epoch should not be changed because the request failed.
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testReadStateFencedLeaderEpochError() { public void testReadStateFencedLeaderEpochError() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
int leaderEpoch = 5; int leaderEpoch = 5;
writeAndReplayRecord(shard, leaderEpoch); // leaderEpoch in the leaderMap will be 5. writeAndReplayRecord(shard, leaderEpoch); // leaderEpoch in the leaderMap will be 5.
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData() .setTopics(List.of(new ReadShareGroupStateRequestData.ReadStateData()
@ -845,7 +872,7 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(leaderEpoch, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(leaderEpoch, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
@ -875,11 +902,11 @@ class ShareCoordinatorShardTest {
// -Share leader acks batch 3 and sends the new startOffset and the state of batch 3 to share coordinator. // -Share leader acks batch 3 and sends the new startOffset and the state of batch 3 to share coordinator.
// -Share coordinator writes the snapshot with startOffset 110 and batch 3. // -Share coordinator writes the snapshot with startOffset 110 and batch 3.
// -batch2 should NOT be lost // -batch2 should NOT be lost
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder() shard = new ShareCoordinatorShardBuilder()
.setConfigOverrides(Map.of(ShareCoordinatorConfig.SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, "0")) .setConfigOverrides(Map.of(ShareCoordinatorConfig.SNAPSHOT_UPDATE_RECORDS_PER_SNAPSHOT_CONFIG, "0"))
.build(); .build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION); initSharePartition(shard, SHARE_PARTITION_KEY);
// Set initial state. // Set initial state.
WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData() WriteShareGroupStateRequestData request = new WriteShareGroupStateRequestData()
@ -916,16 +943,16 @@ class ShareCoordinatorShardTest {
WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); WriteShareGroupStateResponseData expectedData = WriteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), 1, TIME.milliseconds())
)); ));
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), TIME.milliseconds()) GROUP_ID, TOPIC_ID, PARTITION, ShareGroupOffset.fromRequest(request.topics().get(0).partitions().get(0), 1, TIME.milliseconds())
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
// Acknowledge b1. // Acknowledge b1.
@ -977,7 +1004,7 @@ class ShareCoordinatorShardTest {
.setStartOffset(110) .setStartOffset(110)
.setLeaderEpoch(0) .setLeaderEpoch(0)
.setStateEpoch(0) .setStateEpoch(0)
.setSnapshotEpoch(2) // since 2nd share snapshot .setSnapshotEpoch(3) // since 2nd share snapshot
.setStateBatches(List.of( .setStateBatches(List.of(
new PersisterStateBatch(110, 119, (byte) 1, (short) 2), // b2 not lost new PersisterStateBatch(110, 119, (byte) 1, (short) 2), // b2 not lost
new PersisterStateBatch(120, 129, (byte) 2, (short) 1) new PersisterStateBatch(120, 129, (byte) 2, (short) 1)
@ -994,15 +1021,15 @@ class ShareCoordinatorShardTest {
assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( assertEquals(groupOffset(ShareCoordinatorRecordHelpers.newShareSnapshotRecord(
GROUP_ID, TOPIC_ID, PARTITION, offsetFinal GROUP_ID, TOPIC_ID, PARTITION, offsetFinal
).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); ).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(0, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(0, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard(), times(3)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard(), times(3)).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
} }
@Test @Test
public void testLastRedundantOffset() { public void testLastRedundantOffset() {
ShareCoordinatorOffsetsManager manager = mock(ShareCoordinatorOffsetsManager.class); ShareCoordinatorOffsetsManager manager = mock(ShareCoordinatorOffsetsManager.class);
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder() shard = new ShareCoordinatorShardBuilder()
.setOffsetsManager(manager) .setOffsetsManager(manager)
.build(); .build();
@ -1012,9 +1039,7 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testReadStateLeaderEpochUpdateSuccess() { public void testReadStateLeaderEpochUpdateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -1034,7 +1059,7 @@ class ShareCoordinatorShardTest {
PartitionFactory.UNINITIALIZED_START_OFFSET, PartitionFactory.UNINITIALIZED_START_OFFSET,
PartitionFactory.DEFAULT_STATE_EPOCH, PartitionFactory.DEFAULT_STATE_EPOCH,
List.of()); List.of());
List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareSnapshotRecord( List<CoordinatorRecord> expectedRecords = List.of(ShareCoordinatorRecordHelpers.newShareUpdateRecord(
GROUP_ID, TOPIC_ID, PARTITION, new ShareGroupOffset.Builder() GROUP_ID, TOPIC_ID, PARTITION, new ShareGroupOffset.Builder()
.setStartOffset(PartitionFactory.UNINITIALIZED_START_OFFSET) .setStartOffset(PartitionFactory.UNINITIALIZED_START_OFFSET)
.setLeaderEpoch(2) .setLeaderEpoch(2)
@ -1049,14 +1074,14 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertEquals(groupOffset(expectedRecords.get(0).value().message()), shard.getShareStateMapValue(shareCoordinatorKey)); assertEquals(groupOffset(expectedRecords.get(0).value().message()), shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertEquals(2, shard.getLeaderMapValue(shareCoordinatorKey)); assertEquals(2, shard.getLeaderMapValue(SHARE_PARTITION_KEY));
verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME); verify(shard.getMetricsShard()).record(ShareCoordinatorMetrics.SHARE_COORDINATOR_WRITE_SENSOR_NAME);
} }
@Test @Test
public void testReadStateLeaderEpochUpdateNoUpdate() { public void testReadStateLeaderEpochUpdateNoUpdate() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build(); initSharePartition(shard, SHARE_PARTITION_KEY);
ReadShareGroupStateRequestData request1 = new ReadShareGroupStateRequestData() ReadShareGroupStateRequestData request1 = new ReadShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
@ -1103,10 +1128,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testDeleteStateSuccess() { public void testDeleteStateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData() DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new DeleteShareGroupStateRequestData.DeleteStateData() .setTopics(List.of(new DeleteShareGroupStateRequestData.DeleteStateData()
@ -1135,9 +1156,9 @@ class ShareCoordinatorShardTest {
.build() .build()
); );
shard.replay(0L, 0L, (short) 0, record); shard.replay(0L, 0L, (short) 0, record);
assertNotNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNotNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNotNull(shard.getLeaderMapValue(shareCoordinatorKey)); assertNotNull(shard.getLeaderMapValue(SHARE_PARTITION_KEY));
assertNotNull(shard.getStateEpochMapValue(shareCoordinatorKey)); assertNotNull(shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
CoordinatorResult<DeleteShareGroupStateResponseData, CoordinatorRecord> result = shard.deleteState(request); CoordinatorResult<DeleteShareGroupStateResponseData, CoordinatorRecord> result = shard.deleteState(request);
@ -1153,17 +1174,13 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey)); assertNull(shard.getLeaderMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getStateEpochMapValue(shareCoordinatorKey)); assertNull(shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testDeleteStateUnintializedRecord() { public void testDeleteStateUnintializedRecord() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData() DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new DeleteShareGroupStateRequestData.DeleteStateData() .setTopics(List.of(new DeleteShareGroupStateRequestData.DeleteStateData()
@ -1172,10 +1189,10 @@ class ShareCoordinatorShardTest {
.setPartition(PARTITION))))); .setPartition(PARTITION)))));
CoordinatorResult<DeleteShareGroupStateResponseData, CoordinatorRecord> result = shard.deleteState(request); CoordinatorResult<DeleteShareGroupStateResponseData, CoordinatorRecord> result = shard.deleteState(request);
assertNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getLeaderMapValue(shareCoordinatorKey)); assertNull(shard.getLeaderMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getStateEpochMapValue(shareCoordinatorKey)); assertNull(shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
DeleteShareGroupStateResponseData expectedData = DeleteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION); DeleteShareGroupStateResponseData expectedData = DeleteShareGroupStateResponse.toResponseData(TOPIC_ID, PARTITION);
@ -1185,8 +1202,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testDeleteStateInvalidRequestData() { public void testDeleteStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
// invalid partition // invalid partition
int partition = -1; int partition = -1;
@ -1210,7 +1225,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testDeleteNullMetadataImage() { public void testDeleteNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
shard.onNewMetadataImage(null, null); shard.onNewMetadataImage(null, null);
DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData() DeleteShareGroupStateRequestData request = new DeleteShareGroupStateRequestData()
@ -1232,7 +1246,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testDeleteTopicIdNonExistentInMetadataImage() { public void testDeleteTopicIdNonExistentInMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
@ -1264,7 +1277,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testDeletePartitionIdNonExistentInMetadataImage() { public void testDeletePartitionIdNonExistentInMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
@ -1302,10 +1314,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testInitializeStateSuccess() { public void testInitializeStateSuccess() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
SharePartitionKey shareCoordinatorKey = SharePartitionKey.getInstance(GROUP_ID, TOPIC_ID, PARTITION);
InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData() InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData()
.setGroupId(GROUP_ID) .setGroupId(GROUP_ID)
.setTopics(List.of(new InitializeShareGroupStateRequestData.InitializeStateData() .setTopics(List.of(new InitializeShareGroupStateRequestData.InitializeStateData()
@ -1316,8 +1324,8 @@ class ShareCoordinatorShardTest {
.setStateEpoch(5))) .setStateEpoch(5)))
)); ));
assertNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNull(shard.getStateEpochMapValue(shareCoordinatorKey)); assertNull(shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
CoordinatorResult<InitializeShareGroupStateResponseData, CoordinatorRecord> result = shard.initializeState(request); CoordinatorResult<InitializeShareGroupStateResponseData, CoordinatorRecord> result = shard.initializeState(request);
result.records().forEach(record -> shard.replay(0L, 0L, (short) 0, record)); result.records().forEach(record -> shard.replay(0L, 0L, (short) 0, record));
@ -1331,14 +1339,12 @@ class ShareCoordinatorShardTest {
assertEquals(expectedData, result.response()); assertEquals(expectedData, result.response());
assertEquals(expectedRecords, result.records()); assertEquals(expectedRecords, result.records());
assertNotNull(shard.getShareStateMapValue(shareCoordinatorKey)); assertNotNull(shard.getShareStateMapValue(SHARE_PARTITION_KEY));
assertNotNull(shard.getStateEpochMapValue(shareCoordinatorKey)); assertNotNull(shard.getStateEpochMapValue(SHARE_PARTITION_KEY));
} }
@Test @Test
public void testInitializeStateInvalidRequestData() { public void testInitializeStateInvalidRequestData() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
// invalid partition // invalid partition
int partition = -1; int partition = -1;
@ -1391,7 +1397,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testInitializeNullMetadataImage() { public void testInitializeNullMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
shard.onNewMetadataImage(null, null); shard.onNewMetadataImage(null, null);
InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData() InitializeShareGroupStateRequestData request = new InitializeShareGroupStateRequestData()
@ -1415,7 +1420,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testInitializeTopicIdNonExistentInMetadataImage() { public void testInitializeTopicIdNonExistentInMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
@ -1445,7 +1449,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testInitializePartitionIdNonExistentInMetadataImage() { public void testInitializePartitionIdNonExistentInMetadataImage() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
@ -1479,7 +1482,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testSnapshotColdPartitionsNoEligiblePartitions() { public void testSnapshotColdPartitionsNoEligiblePartitions() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
int offset = 0; int offset = 0;
@ -1546,7 +1548,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() { public void testSnapshotColdPartitionsSnapshotUpdateNotConsidered() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
int offset = 0; int offset = 0;
@ -1644,7 +1645,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() { public void testSnapshotColdPartitionsDoesNotPerpetuallySnapshot() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
int offset = 0; int offset = 0;
@ -1719,7 +1719,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testSnapshotColdPartitionsPartialEligiblePartitions() { public void testSnapshotColdPartitionsPartialEligiblePartitions() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
int offset = 0; int offset = 0;
@ -1824,8 +1823,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testOnTopicsDeletedEmptyTopicIds() { public void testOnTopicsDeletedEmptyTopicIds() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
CoordinatorResult<Void, CoordinatorRecord> expectedResult = new CoordinatorResult<>(List.of()); CoordinatorResult<Void, CoordinatorRecord> expectedResult = new CoordinatorResult<>(List.of());
assertEquals(expectedResult, shard.maybeCleanupShareState(Set.of())); assertEquals(expectedResult, shard.maybeCleanupShareState(Set.of()));
@ -1836,7 +1833,6 @@ class ShareCoordinatorShardTest {
@Test @Test
public void testOnTopicsDeletedTopicIds() { public void testOnTopicsDeletedTopicIds() {
ShareCoordinatorShard shard = new ShareCoordinatorShardBuilder().build();
MetadataImage image = mock(MetadataImage.class); MetadataImage image = mock(MetadataImage.class);
shard.onNewMetadataImage(image, null); shard.onNewMetadataImage(image, null);
@ -1909,4 +1905,20 @@ class ShareCoordinatorShardTest {
} }
return ShareGroupOffset.fromRecord((ShareUpdateValue) record); return ShareGroupOffset.fromRecord((ShareUpdateValue) record);
} }
private void initSharePartition(ShareCoordinatorShard shard, SharePartitionKey key) {
shard.replay(0L, 0L, (short) 0, CoordinatorRecord.record(
new ShareSnapshotKey()
.setGroupId(key.groupId())
.setTopicId(key.topicId())
.setPartition(key.partition()),
new ApiMessageAndVersion(
new ShareSnapshotValue()
.setStateEpoch(0)
.setLeaderEpoch(-1)
.setStartOffset(-1),
(short) 0
)
));
}
} }