chia7712 commented on code in PR #19820:
URL: https://github.com/apache/kafka/pull/19820#discussion_r2217941355
##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java:
##########
@@ -366,6 +378,75 @@ Entry<Throwable, Map<String, Throwable>>
sendDeleteShareGroupOffsetsRequest(Stri
return new SimpleImmutableEntry<>(topLevelException,
topicLevelResult);
}
+ void resetOffsets() {
+ String groupId = opts.options.valueOf(opts.groupOpt);
+ try {
+ ShareGroupDescription shareGroupDescription =
describeShareGroups(List.of(groupId)).get(groupId);
+ if
(!GroupState.EMPTY.equals(shareGroupDescription.groupState())) {
Review Comment:
Should `DEAD` state be included?
##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -709,15 +709,47 @@ CompletableFuture<AlterShareGroupOffsetsResponseData>
persisterInitialize(
handlePersisterInitializeResponse(request.groupTopicPartitionData().groupId(),
result, new ShareGroupHeartbeatResponseData());
return response;
} else {
- //TODO build new AlterShareGroupOffsetsResponseData
for error response
- return response;
+ return buildErrorResponse(response, result);
}
} else {
return buildErrorResponse(request, response, exp);
}
});
}
+
+ private AlterShareGroupOffsetsResponseData
buildErrorResponse(AlterShareGroupOffsetsResponseData response,
InitializeShareGroupStateResult result) {
+ AlterShareGroupOffsetsResponseData data = new
AlterShareGroupOffsetsResponseData();
+ data.setResponses(
+ new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopicCollection(response.responses().stream()
+ .map(topic -> {
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic
topicData = new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
+ .setTopicName(topic.topicName());
+ Map<Uuid, Map<Integer, PartitionErrorData>>
topicPartitionErrorsMap = result.getErrors();
Review Comment:
Could you please avoid creating the `topicPartitionErrorsMap` for each loop
(`topic`)?
##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -709,15 +709,47 @@ CompletableFuture<AlterShareGroupOffsetsResponseData>
persisterInitialize(
handlePersisterInitializeResponse(request.groupTopicPartitionData().groupId(),
result, new ShareGroupHeartbeatResponseData());
return response;
} else {
- //TODO build new AlterShareGroupOffsetsResponseData
for error response
- return response;
+ return buildErrorResponse(response, result);
}
} else {
return buildErrorResponse(request, response, exp);
}
});
}
+
+ private AlterShareGroupOffsetsResponseData
buildErrorResponse(AlterShareGroupOffsetsResponseData response,
InitializeShareGroupStateResult result) {
+ AlterShareGroupOffsetsResponseData data = new
AlterShareGroupOffsetsResponseData();
+ data.setResponses(
+ new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopicCollection(response.responses().stream()
+ .map(topic -> {
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic
topicData = new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
+ .setTopicName(topic.topicName());
+ Map<Uuid, Map<Integer, PartitionErrorData>>
topicPartitionErrorsMap = result.getErrors();
+ topic.partitions().forEach(partition -> {
+ if (partition.errorCode() != Errors.NONE.code()) {
+ topicData.partitions().add(partition);
+ return;
+ }
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition
partitionData;
+ PartitionErrorData error =
topicPartitionErrorsMap.get(topic.topicId()).get(partition.partitionIndex());
Review Comment:
should we check the null for `topicPartitionErrorsMap.get(topic.topicId())`?
##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -709,15 +709,47 @@ CompletableFuture<AlterShareGroupOffsetsResponseData>
persisterInitialize(
handlePersisterInitializeResponse(request.groupTopicPartitionData().groupId(),
result, new ShareGroupHeartbeatResponseData());
return response;
} else {
- //TODO build new AlterShareGroupOffsetsResponseData
for error response
- return response;
+ return buildErrorResponse(response, result);
}
} else {
return buildErrorResponse(request, response, exp);
}
});
}
+
+ private AlterShareGroupOffsetsResponseData
buildErrorResponse(AlterShareGroupOffsetsResponseData response,
InitializeShareGroupStateResult result) {
+ AlterShareGroupOffsetsResponseData data = new
AlterShareGroupOffsetsResponseData();
+ data.setResponses(
+ new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopicCollection(response.responses().stream()
+ .map(topic -> {
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic
topicData = new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
+ .setTopicName(topic.topicName());
Review Comment:
not sure why the topic id is not set for the response
##########
group-coordinator/src/main/java/org/apache/kafka/coordinator/group/GroupCoordinatorService.java:
##########
@@ -709,15 +709,47 @@ CompletableFuture<AlterShareGroupOffsetsResponseData>
persisterInitialize(
handlePersisterInitializeResponse(request.groupTopicPartitionData().groupId(),
result, new ShareGroupHeartbeatResponseData());
return response;
} else {
- //TODO build new AlterShareGroupOffsetsResponseData
for error response
- return response;
+ return buildErrorResponse(response, result);
}
} else {
return buildErrorResponse(request, response, exp);
}
});
}
+
+ private AlterShareGroupOffsetsResponseData
buildErrorResponse(AlterShareGroupOffsetsResponseData response,
InitializeShareGroupStateResult result) {
+ AlterShareGroupOffsetsResponseData data = new
AlterShareGroupOffsetsResponseData();
+ data.setResponses(
+ new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopicCollection(response.responses().stream()
+ .map(topic -> {
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic
topicData = new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponseTopic()
+ .setTopicName(topic.topicName());
+ Map<Uuid, Map<Integer, PartitionErrorData>>
topicPartitionErrorsMap = result.getErrors();
+ topic.partitions().forEach(partition -> {
+ if (partition.errorCode() != Errors.NONE.code()) {
+ topicData.partitions().add(partition);
+ return;
+ }
+
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition
partitionData;
+ PartitionErrorData error =
topicPartitionErrorsMap.get(topic.topicId()).get(partition.partitionIndex());
+ if (error == null) {
+ partitionData = new
AlterShareGroupOffsetsResponseData.AlterShareGroupOffsetsResponsePartition()
Review Comment:
we could reuse the `partition` instead of creating new
`AlterShareGroupOffsetsResponsePartition`, right?
##########
tools/src/main/java/org/apache/kafka/tools/consumer/group/ShareGroupCommand.java:
##########
@@ -366,6 +378,75 @@ Entry<Throwable, Map<String, Throwable>>
sendDeleteShareGroupOffsetsRequest(Stri
return new SimpleImmutableEntry<>(topLevelException,
topicLevelResult);
}
+ void resetOffsets() {
+ String groupId = opts.options.valueOf(opts.groupOpt);
+ try {
+ ShareGroupDescription shareGroupDescription =
describeShareGroups(List.of(groupId)).get(groupId);
+ if
(!GroupState.EMPTY.equals(shareGroupDescription.groupState())) {
+ CommandLineUtils.printErrorAndExit(String.format("Share
group '%s' is not empty.", groupId));
+ }
+ Map<TopicPartition, OffsetAndMetadata> offsetsToReset =
prepareOffsetsToReset(groupId);
+ if (offsetsToReset == null) {
+ return;
+ }
+ boolean dryRun = opts.options.has(opts.dryRunOpt) ||
!opts.options.has(opts.executeOpt);
+ if (!dryRun) {
+ adminClient.alterShareGroupOffsets(groupId,
+ offsetsToReset.entrySet().stream()
+ .collect(Collectors.toMap(
+ Entry::getKey, entry ->
entry.getValue().offset()
+ ))
+ ).all().get();
+ }
+ OffsetsUtils.printOffsetsToReset(Map.of(groupId,
offsetsToReset));
+ } catch (InterruptedException ie) {
+ throw new RuntimeException(ie);
+ } catch (ExecutionException ee) {
+ Throwable cause = ee.getCause();
+ if (cause instanceof KafkaException) {
+ CommandLineUtils.printErrorAndExit(cause.getMessage());
+ } else {
+ throw new RuntimeException(cause);
+ }
+ }
+ }
+
+ protected Map<TopicPartition, OffsetAndMetadata>
prepareOffsetsToReset(String groupId) throws ExecutionException,
InterruptedException {
+ Map<String, ListShareGroupOffsetsSpec> groupSpecs =
Map.of(groupId, new ListShareGroupOffsetsSpec());
+ Map<TopicPartition, OffsetAndMetadata> offsetsByTopicPartitions =
adminClient.listShareGroupOffsets(groupSpecs).all().get().get(groupId);
+ Set<String> existsTopics = adminClient.listTopics().names().get();
+
+ if (opts.options.has(opts.topicOpt)) {
Review Comment:
This behavior is not consistent with normal consumer tool. It should include
following tasks.
1. parse the string for "topic" and "topic partition". It means both
`--topicc a` and `--topic a:0` are valid
2. parse the `allTopicsOpt`
3. if `allTopicsOpt` is defined, all topics in the group should be reset.
4. if `topicOpt` is defined, all specified tps should be reset
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]