Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

KAFKA-16750: Added acknowledge code in SharePartitionManager including unit tests #16457

Merged
merged 4 commits into from
Jul 2, 2024
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
1 change: 1 addition & 0 deletions checkstyle/suppressions.xml
Original file line number Diff line number Diff line change
Expand Up @@ -45,6 +45,7 @@
files="(KafkaClusterTestKit).java"/>
<suppress checks="JavaNCSS"
files="RemoteLogManagerTest.java"/>
<suppress checks="ClassDataAbstractionCoupling" files="SharePartitionManagerTest"/>
chirag-wadhwa5 marked this conversation as resolved.
Show resolved Hide resolved

<!-- server tests -->
<suppress checks="MethodLength|JavaNCSS|NPath" files="DescribeTopicPartitionsRequestHandlerTest.java"/>
Expand Down
30 changes: 26 additions & 4 deletions core/src/main/java/kafka/server/share/SharePartitionManager.java
Original file line number Diff line number Diff line change
Expand Up @@ -252,11 +252,33 @@ public CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.Part
) {
log.trace("Acknowledge request for topicIdPartitions: {} with groupId: {}",
acknowledgeTopics.keySet(), groupId);
Map<TopicIdPartition, CompletableFuture<Errors>> futures = new HashMap<>();
acknowledgeTopics.forEach((topicIdPartition, acknowledgePartitionBatches) -> {
SharePartition sharePartition = partitionCacheMap.get(sharePartitionKey(groupId, topicIdPartition));
if (sharePartition != null) {
CompletableFuture<Errors> future = sharePartition.acknowledge(memberId, acknowledgePartitionBatches).thenApply(throwable -> {
if (throwable.isPresent()) {
return Errors.forException(throwable.get());
}
return Errors.NONE;
});
futures.put(topicIdPartition, future);
} else {
futures.put(topicIdPartition, CompletableFuture.completedFuture(Errors.UNKNOWN_TOPIC_OR_PARTITION));
}
});

CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> future = new CompletableFuture<>();
future.completeExceptionally(new UnsupportedOperationException("Not implemented yet"));

return future;
CompletableFuture<Void> allFutures = CompletableFuture.allOf(
futures.values().toArray(new CompletableFuture[0]));
return allFutures.thenApply(v -> {
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = new HashMap<>();
futures.forEach((topicIdPartition, future) -> {
result.put(topicIdPartition, new ShareAcknowledgeResponseData.PartitionData()
.setPartitionIndex(topicIdPartition.partition())
.setErrorCode(future.join().code()));
});
return result;
});
}

/**
Expand Down
163 changes: 163 additions & 0 deletions core/src/test/java/kafka/server/share/SharePartitionManagerTest.java
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,7 @@
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.server.group.share.NoOpShareStatePersister;
import org.apache.kafka.server.group.share.Persister;
import org.apache.kafka.server.share.ShareAcknowledgementBatch;
import org.apache.kafka.server.share.ShareSessionCache;
import org.apache.kafka.server.share.ShareSessionKey;
import org.apache.kafka.server.util.timer.MockTimer;
Expand Down Expand Up @@ -1364,6 +1365,168 @@ public void testReleaseAcquiredRecordsWithEmptyTopicPartitions() {
assertEquals(0, result.size());
}

@Test
public void testAcknowledgeSinglePartition() {
String groupId = "grp";
String memberId = Uuid.randomUuid().toString();

TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
SharePartition sp = mock(SharePartition.class);

when(sp.acknowledge(ArgumentMatchers.eq(memberId), any())).thenReturn(CompletableFuture.completedFuture(Optional.empty()));

Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp), sp);
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
.withPartitionCacheMap(partitionCacheMap).build();

Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
acknowledgeTopics.put(tp, Arrays.asList(
new ShareAcknowledgementBatch(12, 20, Collections.singletonList((byte) 1)),
new ShareAcknowledgementBatch(24, 56, Collections.singletonList((byte) 1))
));
CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> resultFuture =
sharePartitionManager.acknowledge(memberId, groupId, acknowledgeTopics);
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = resultFuture.join();
assertEquals(1, result.size());
assertTrue(result.containsKey(tp));
assertEquals(0, result.get(tp).partitionIndex());
assertEquals(Errors.NONE.code(), result.get(tp).errorCode());
}

@Test
public void testAcknowledgeMultiplePartition() {
String groupId = "grp";
String memberId = Uuid.randomUuid().toString();

TopicIdPartition tp1 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo1", 0));
TopicIdPartition tp2 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo2", 0));
TopicIdPartition tp3 = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo3", 0));

SharePartition sp1 = mock(SharePartition.class);
SharePartition sp2 = mock(SharePartition.class);
SharePartition sp3 = mock(SharePartition.class);

when(sp1.acknowledge(ArgumentMatchers.eq(memberId), any())).thenReturn(CompletableFuture.completedFuture(Optional.empty()));
when(sp2.acknowledge(ArgumentMatchers.eq(memberId), any())).thenReturn(CompletableFuture.completedFuture(Optional.empty()));
when(sp3.acknowledge(ArgumentMatchers.eq(memberId), any())).thenReturn(CompletableFuture.completedFuture(Optional.empty()));

Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp1), sp1);
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp2), sp2);
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp3), sp3);

Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
chirag-wadhwa5 marked this conversation as resolved.
Show resolved Hide resolved

SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
.withPartitionCacheMap(partitionCacheMap).build();
acknowledgeTopics.put(tp1, Arrays.asList(
new ShareAcknowledgementBatch(12, 20, Collections.singletonList((byte) 1)),
new ShareAcknowledgementBatch(24, 56, Collections.singletonList((byte) 1))
));
acknowledgeTopics.put(tp2, Arrays.asList(
new ShareAcknowledgementBatch(15, 26, Collections.singletonList((byte) 2)),
new ShareAcknowledgementBatch(34, 56, Collections.singletonList((byte) 2))
));
acknowledgeTopics.put(tp3, Arrays.asList(
new ShareAcknowledgementBatch(4, 15, Collections.singletonList((byte) 3)),
new ShareAcknowledgementBatch(16, 21, Collections.singletonList((byte) 3))
));
CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> resultFuture =
sharePartitionManager.acknowledge(memberId, groupId, acknowledgeTopics);
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = resultFuture.join();
assertEquals(3, result.size());
assertTrue(result.containsKey(tp1));
assertTrue(result.containsKey(tp2));
assertTrue(result.containsKey(tp3));
assertEquals(0, result.get(tp1).partitionIndex());
assertEquals(Errors.NONE.code(), result.get(tp1).errorCode());
assertEquals(0, result.get(tp2).partitionIndex());
assertEquals(Errors.NONE.code(), result.get(tp2).errorCode());
assertEquals(0, result.get(tp3).partitionIndex());
assertEquals(Errors.NONE.code(), result.get(tp3).errorCode());
}

@Test
public void testAcknowledgeIncorrectGroupId() {
String groupId = "grp";
String groupId2 = "grp2";
String memberId = Uuid.randomUuid().toString();

TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
SharePartition sp = mock(SharePartition.class);

Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp), sp);
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
.withPartitionCacheMap(partitionCacheMap).build();

Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
acknowledgeTopics.put(tp, Arrays.asList(
new ShareAcknowledgementBatch(12, 20, Collections.singletonList((byte) 1)),
new ShareAcknowledgementBatch(24, 56, Collections.singletonList((byte) 1))
));
CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> resultFuture =
sharePartitionManager.acknowledge(memberId, groupId2, acknowledgeTopics);
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = resultFuture.join();
assertEquals(1, result.size());
assertTrue(result.containsKey(tp));
assertEquals(0, result.get(tp).partitionIndex());
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), result.get(tp).errorCode());
}

@Test
public void testAcknowledgeIncorrectMemberId() {
String groupId = "grp";
String memberId = Uuid.randomUuid().toString();

TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo", 0));
SharePartition sp = mock(SharePartition.class);
when(sp.acknowledge(ArgumentMatchers.eq(memberId), any())).thenReturn(CompletableFuture.completedFuture(
Optional.of(new InvalidRequestException("Member is not the owner of batch record"))
));
Map<SharePartitionManager.SharePartitionKey, SharePartition> partitionCacheMap = new HashMap<>();
partitionCacheMap.put(new SharePartitionManager.SharePartitionKey(groupId, tp), sp);
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder()
.withPartitionCacheMap(partitionCacheMap).build();

Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
acknowledgeTopics.put(tp, Arrays.asList(
new ShareAcknowledgementBatch(12, 20, Collections.singletonList((byte) 1)),
new ShareAcknowledgementBatch(24, 56, Collections.singletonList((byte) 1))
));

CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> resultFuture =
sharePartitionManager.acknowledge(memberId, groupId, acknowledgeTopics);
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = resultFuture.join();
assertEquals(1, result.size());
assertTrue(result.containsKey(tp));
assertEquals(0, result.get(tp).partitionIndex());
assertEquals(Errors.INVALID_REQUEST.code(), result.get(tp).errorCode());
}

@Test
public void testAcknowledgeEmptyPartitionCacheMap() {
String groupId = "grp";
String memberId = Uuid.randomUuid().toString();

TopicIdPartition tp = new TopicIdPartition(Uuid.randomUuid(), new TopicPartition("foo4", 3));
SharePartitionManager sharePartitionManager = SharePartitionManagerBuilder.builder().build();

Map<TopicIdPartition, List<ShareAcknowledgementBatch>> acknowledgeTopics = new HashMap<>();
acknowledgeTopics.put(tp, Arrays.asList(
new ShareAcknowledgementBatch(78, 90, Collections.singletonList((byte) 2)),
new ShareAcknowledgementBatch(94, 99, Collections.singletonList((byte) 2))
));
CompletableFuture<Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData>> resultFuture =
sharePartitionManager.acknowledge(memberId, groupId, acknowledgeTopics);
Map<TopicIdPartition, ShareAcknowledgeResponseData.PartitionData> result = resultFuture.join();
assertEquals(1, result.size());
assertTrue(result.containsKey(tp));
assertEquals(3, result.get(tp).partitionIndex());
assertEquals(Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), result.get(tp).errorCode());
}

private ShareFetchResponseData.PartitionData noErrorShareFetchResponse() {
return new ShareFetchResponseData.PartitionData().setPartitionIndex(0);
}
Expand Down