|
28 | 28 | import org.apache.kafka.common.message.DeleteGroupsResponseData;
|
29 | 29 | import org.apache.kafka.common.message.DeleteShareGroupOffsetsRequestData;
|
30 | 30 | import org.apache.kafka.common.message.DeleteShareGroupOffsetsResponseData;
|
31 |
| -import org.apache.kafka.common.message.DeleteShareGroupStateRequestData; |
32 | 31 | import org.apache.kafka.common.message.DescribeGroupsResponseData;
|
33 | 32 | import org.apache.kafka.common.message.DescribeShareGroupOffsetsRequestData;
|
34 | 33 | import org.apache.kafka.common.message.DescribeShareGroupOffsetsResponseData;
|
@@ -1258,45 +1257,40 @@ private CompletableFuture<Map<String, Errors>> persisterDeleteToGroupIdErrorMap(
|
1258 | 1257 | });
|
1259 | 1258 | }
|
1260 | 1259 |
|
1261 |
| - private void populateDeleteShareGroupOffsetsFuture( |
1262 |
| - DeleteShareGroupOffsetsRequestData requestData, |
1263 |
| - CompletableFuture<DeleteShareGroupOffsetsResponseData> future, |
1264 |
| - Map<Uuid, String> requestTopicIdToNameMapping, |
1265 |
| - List<DeleteShareGroupStateRequestData.DeleteStateData> deleteShareGroupStateRequestTopicsData, |
1266 |
| - List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> deleteShareGroupOffsetsResponseTopicList |
1267 |
| - |
| 1260 | + private CompletableFuture<DeleteShareGroupOffsetsResponseData> persistDeleteShareGroupOffsets( |
| 1261 | + DeleteShareGroupStateParameters deleteStateRequestParameters, |
| 1262 | + List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> errorTopicResponseList |
1268 | 1263 | ) {
|
1269 |
| - DeleteShareGroupStateRequestData deleteShareGroupStateRequestData = new DeleteShareGroupStateRequestData() |
1270 |
| - .setGroupId(requestData.groupId()) |
1271 |
| - .setTopics(deleteShareGroupStateRequestTopicsData); |
1272 |
| - |
1273 |
| - persister.deleteState(DeleteShareGroupStateParameters.from(deleteShareGroupStateRequestData)) |
1274 |
| - .whenComplete((result, error) -> { |
1275 |
| - if (error != null) { |
1276 |
| - log.error("Failed to delete share group state"); |
1277 |
| - future.completeExceptionally(error); |
1278 |
| - return; |
1279 |
| - } |
| 1264 | + return persister.deleteState(deleteStateRequestParameters) |
| 1265 | + .thenCompose(result -> { |
1280 | 1266 | if (result == null || result.topicsData() == null) {
|
1281 | 1267 | log.error("Result is null for the delete share group state");
|
1282 |
| - future.completeExceptionally(new IllegalStateException("Result is null for the delete share group state")); |
1283 |
| - return; |
| 1268 | + Exception exception = new IllegalStateException("Result is null for the delete share group state"); |
| 1269 | + return CompletableFuture.completedFuture( |
| 1270 | + DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(exception)) |
| 1271 | + ); |
1284 | 1272 | }
|
1285 | 1273 | result.topicsData().forEach(topicData ->
|
1286 |
| - deleteShareGroupOffsetsResponseTopicList.add(new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic() |
1287 |
| - .setTopicId(topicData.topicId()) |
1288 |
| - .setTopicName(requestTopicIdToNameMapping.get(topicData.topicId())) |
1289 |
| - .setPartitions(topicData.partitions().stream().map( |
1290 |
| - partitionData -> new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition() |
1291 |
| - .setPartitionIndex(partitionData.partition()) |
1292 |
| - .setErrorMessage(partitionData.errorCode() == Errors.NONE.code() ? null : Errors.forCode(partitionData.errorCode()).message()) |
1293 |
| - .setErrorCode(partitionData.errorCode()) |
1294 |
| - ).toList()) |
1295 |
| - )); |
| 1274 | + errorTopicResponseList.add( |
| 1275 | + new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic() |
| 1276 | + .setTopicId(topicData.topicId()) |
| 1277 | + .setTopicName(metadataImage.topics().topicIdToNameView().get(topicData.topicId())) |
| 1278 | + .setPartitions(topicData.partitions().stream().map( |
| 1279 | + partitionData -> new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition() |
| 1280 | + .setPartitionIndex(partitionData.partition()) |
| 1281 | + .setErrorMessage(partitionData.errorCode() == Errors.NONE.code() ? null : Errors.forCode(partitionData.errorCode()).message()) |
| 1282 | + .setErrorCode(partitionData.errorCode()) |
| 1283 | + ).toList()) |
| 1284 | + ) |
| 1285 | + ); |
1296 | 1286 |
|
1297 |
| - future.complete( |
| 1287 | + return CompletableFuture.completedFuture( |
1298 | 1288 | new DeleteShareGroupOffsetsResponseData()
|
1299 |
| - .setResponses(deleteShareGroupOffsetsResponseTopicList)); |
| 1289 | + .setResponses(errorTopicResponseList) |
| 1290 | + ); |
| 1291 | + }).exceptionally(throwable -> { |
| 1292 | + log.error("Failed to delete share group state"); |
| 1293 | + return DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(throwable)); |
1300 | 1294 | });
|
1301 | 1295 | }
|
1302 | 1296 |
|
@@ -1590,83 +1584,53 @@ public CompletableFuture<DeleteShareGroupOffsetsResponseData> deleteShareGroupOf
|
1590 | 1584 | DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.INVALID_GROUP_ID));
|
1591 | 1585 | }
|
1592 | 1586 |
|
1593 |
| - Map<Uuid, String> requestTopicIdToNameMapping = new HashMap<>(); |
1594 |
| - List<DeleteShareGroupStateRequestData.DeleteStateData> deleteShareGroupStateRequestTopicsData = new ArrayList<>(requestData.topics().size()); |
1595 |
| - List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> deleteShareGroupOffsetsResponseTopicList = new ArrayList<>(requestData.topics().size()); |
1596 |
| - |
1597 |
| - requestData.topics().forEach(topic -> { |
1598 |
| - Uuid topicId = metadataImage.topics().topicNameToIdView().get(topic.topicName()); |
1599 |
| - if (topicId != null) { |
1600 |
| - requestTopicIdToNameMapping.put(topicId, topic.topicName()); |
1601 |
| - deleteShareGroupStateRequestTopicsData.add(new DeleteShareGroupStateRequestData.DeleteStateData() |
1602 |
| - .setTopicId(topicId) |
1603 |
| - .setPartitions( |
1604 |
| - topic.partitions().stream().map( |
1605 |
| - partitionIndex -> new DeleteShareGroupStateRequestData.PartitionData().setPartition(partitionIndex) |
1606 |
| - ).toList() |
1607 |
| - )); |
1608 |
| - } else { |
1609 |
| - deleteShareGroupOffsetsResponseTopicList.add(new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic() |
1610 |
| - .setTopicName(topic.topicName()) |
1611 |
| - .setPartitions(topic.partitions().stream().map( |
1612 |
| - partition -> new DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponsePartition() |
1613 |
| - .setPartitionIndex(partition) |
1614 |
| - .setErrorCode(Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) |
1615 |
| - .setErrorMessage(Errors.UNKNOWN_TOPIC_OR_PARTITION.message()) |
1616 |
| - ).toList())); |
1617 |
| - } |
1618 |
| - }); |
1619 |
| - |
1620 |
| - // If the request for the persister is empty, just complete the operation right away. |
1621 |
| - if (deleteShareGroupStateRequestTopicsData.isEmpty()) { |
| 1587 | + if (requestData.topics() == null || requestData.topics().isEmpty()) { |
1622 | 1588 | return CompletableFuture.completedFuture(
|
1623 | 1589 | new DeleteShareGroupOffsetsResponseData()
|
1624 |
| - .setResponses(deleteShareGroupOffsetsResponseTopicList)); |
| 1590 | + ); |
1625 | 1591 | }
|
1626 | 1592 |
|
1627 |
| - CompletableFuture<DeleteShareGroupOffsetsResponseData> future = new CompletableFuture<>(); |
| 1593 | + return runtime.scheduleReadOperation( |
| 1594 | + "share-group-delete-offsets-request", |
| 1595 | + topicPartitionFor(groupId), |
| 1596 | + (coordinator, lastCommittedOffset) -> coordinator.shareGroupDeleteOffsetsRequest(groupId, requestData) |
| 1597 | + ) |
| 1598 | + .thenCompose(resultHolder -> { |
| 1599 | + if (resultHolder == null) { |
| 1600 | + log.error("Failed to retrieve deleteState request parameters from group coordinator for the group {}", groupId); |
| 1601 | + return CompletableFuture.completedFuture( |
| 1602 | + DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.UNKNOWN_SERVER_ERROR) |
| 1603 | + ); |
| 1604 | + } |
| 1605 | + |
| 1606 | + if (resultHolder.topLevelErrorCode() != Errors.NONE.code()) { |
| 1607 | + return CompletableFuture.completedFuture( |
| 1608 | + DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData( |
| 1609 | + resultHolder.topLevelErrorCode(), |
| 1610 | + resultHolder.topLevelErrorMessage() |
| 1611 | + ) |
| 1612 | + ); |
| 1613 | + } |
1628 | 1614 |
|
1629 |
| - TopicPartition topicPartition = topicPartitionFor(groupId); |
| 1615 | + List<DeleteShareGroupOffsetsResponseData.DeleteShareGroupOffsetsResponseTopic> errorTopicResponseList = |
| 1616 | + resultHolder.errorTopicResponseList() == null ? new ArrayList<>() : new ArrayList<>(resultHolder.errorTopicResponseList()); |
1630 | 1617 |
|
1631 |
| - // This is done to make sure the provided group is empty. Offsets can be deleted only for an empty share group. |
1632 |
| - CompletableFuture<List<ShareGroupDescribeResponseData.DescribedGroup>> describeGroupFuture = |
1633 |
| - runtime.scheduleReadOperation( |
1634 |
| - "share-group-describe", |
1635 |
| - topicPartition, |
1636 |
| - (coordinator, lastCommittedOffset) -> coordinator.shareGroupDescribe(List.of(groupId), lastCommittedOffset) |
1637 |
| - ).exceptionally(exception -> handleOperationException( |
1638 |
| - "share-group-describe", |
1639 |
| - List.of(groupId), |
1640 |
| - exception, |
1641 |
| - (error, __) -> ShareGroupDescribeRequest.getErrorDescribedGroupList(List.of(groupId), error), |
1642 |
| - log |
1643 |
| - )); |
| 1618 | + if (resultHolder.deleteStateRequestParameters() == null) { |
| 1619 | + return CompletableFuture.completedFuture( |
| 1620 | + new DeleteShareGroupOffsetsResponseData() |
| 1621 | + .setResponses(errorTopicResponseList) |
| 1622 | + ); |
| 1623 | + } |
1644 | 1624 |
|
1645 |
| - describeGroupFuture.whenComplete((groups, throwable) -> { |
1646 |
| - if (throwable != null) { |
1647 |
| - log.error("Failed to describe the share group {}", groupId, throwable); |
1648 |
| - future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(throwable))); |
1649 |
| - } else if (groups == null || groups.isEmpty()) { |
1650 |
| - log.error("Describe share group resulted in empty response for group {}", groupId); |
1651 |
| - future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.GROUP_ID_NOT_FOUND)); |
1652 |
| - } else if (groups.get(0).errorCode() != Errors.NONE.code()) { |
1653 |
| - log.error("Failed to describe the share group {}", groupId); |
1654 |
| - future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(groups.get(0).errorCode(), groups.get(0).errorMessage())); |
1655 |
| - } else if (groups.get(0).members() != null && !groups.get(0).members().isEmpty()) { |
1656 |
| - log.error("Provided group {} is not empty", groupId); |
1657 |
| - future.complete(DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.NON_EMPTY_GROUP)); |
1658 |
| - } else { |
1659 |
| - populateDeleteShareGroupOffsetsFuture( |
1660 |
| - requestData, |
1661 |
| - future, |
1662 |
| - requestTopicIdToNameMapping, |
1663 |
| - deleteShareGroupStateRequestTopicsData, |
1664 |
| - deleteShareGroupOffsetsResponseTopicList |
| 1625 | + return persistDeleteShareGroupOffsets( |
| 1626 | + resultHolder.deleteStateRequestParameters(), |
| 1627 | + errorTopicResponseList |
1665 | 1628 | );
|
1666 |
| - } |
1667 |
| - }); |
1668 |
| - |
1669 |
| - return future; |
| 1629 | + }) |
| 1630 | + .exceptionally(throwable -> { |
| 1631 | + log.error("Failed to retrieve deleteState request parameters from group coordinator for the group {}", groupId, throwable); |
| 1632 | + return DeleteShareGroupOffsetsRequest.getErrorDeleteResponseData(Errors.forException(throwable)); |
| 1633 | + }); |
1670 | 1634 | }
|
1671 | 1635 |
|
1672 | 1636 | /**
|
|
0 commit comments