-
Notifications
You must be signed in to change notification settings - Fork 15.2k
KAFKA-20524: CSV reset offset plan for kafka-share-groups.sh #22197
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
base: trunk
Are you sure you want to change the base?
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -46,6 +46,9 @@ | |
| import org.apache.kafka.server.util.CommandLineUtils; | ||
| import org.apache.kafka.tools.OffsetsUtils; | ||
|
|
||
| import com.fasterxml.jackson.core.JsonProcessingException; | ||
| import com.fasterxml.jackson.databind.ObjectWriter; | ||
|
|
||
| import java.io.IOException; | ||
| import java.util.AbstractMap.SimpleImmutableEntry; | ||
| import java.util.ArrayList; | ||
|
|
@@ -98,7 +101,13 @@ public static void run(ShareGroupCommandOptions opts) { | |
| } else if (opts.options.has(opts.deleteOpt)) { | ||
| shareGroupService.deleteShareGroups(); | ||
| } else if (opts.options.has(opts.resetOffsetsOpt)) { | ||
| shareGroupService.resetOffsets(); | ||
| Map<String, Map<TopicPartition, OffsetAndMetadata>> offsetsToReset = shareGroupService.resetOffsets(); | ||
| if (opts.options.has(opts.exportOpt)) { | ||
| String exported = shareGroupService.exportOffsetsToCsv(offsetsToReset); | ||
| System.out.println(exported); | ||
| } else { | ||
| OffsetsUtils.printOffsetsToReset(offsetsToReset); | ||
| } | ||
| } else if (opts.options.has(opts.deleteOffsetsOpt)) { | ||
| shareGroupService.deleteOffsets(); | ||
| } | ||
|
|
@@ -150,6 +159,8 @@ public ShareGroupService(ShareGroupCommandOptions opts, Admin adminClient) { | |
| private OffsetsUtils.OffsetsUtilsOptions getOffsetsUtilsOptions(ShareGroupCommandOptions opts) { | ||
| return | ||
| new OffsetsUtils.OffsetsUtilsOptions(opts.options.valuesOf(opts.groupOpt), | ||
| opts.options.valuesOf(opts.resetToOffsetOpt), | ||
| opts.options.valuesOf(opts.resetFromFileOpt), | ||
| opts.options.valuesOf(opts.resetToDatetimeOpt), | ||
| opts.options.valueOf(opts.timeoutMsOpt)); | ||
| } | ||
|
|
@@ -242,6 +253,26 @@ public void describeGroups() throws ExecutionException, InterruptedException { | |
| } | ||
| } | ||
|
|
||
| String exportOffsetsToCsv(Map<String, Map<TopicPartition, OffsetAndMetadata>> offsetsForGroups) { | ||
| ObjectWriter csvWriter = CsvUtils.writerFor(CsvUtils.CsvRecordNoGroup.class); | ||
|
|
||
| return offsetsForGroups.entrySet().stream().flatMap(e -> { | ||
| Map<TopicPartition, OffsetAndMetadata> partitionInfo = e.getValue(); | ||
|
|
||
| return partitionInfo.entrySet().stream().map(e1 -> { | ||
| TopicPartition k = e1.getKey(); | ||
| OffsetAndMetadata v = e1.getValue(); | ||
| Object csvRecord = new CsvUtils.CsvRecordNoGroup(k.topic(), k.partition(), v.offset()); | ||
|
|
||
| try { | ||
| return csvWriter.writeValueAsString(csvRecord); | ||
| } catch (JsonProcessingException err) { | ||
| throw new RuntimeException(err); | ||
| } | ||
| }); | ||
| }).collect(Collectors.joining()); | ||
| } | ||
|
|
||
| Map<String, Throwable> deleteShareGroups() { | ||
| List<GroupListing> shareGroupIds = listDetailedShareGroups(); | ||
| List<String> groupIds = opts.options.has(opts.allGroupsOpt) | ||
|
|
@@ -382,32 +413,36 @@ Entry<Throwable, Map<String, Throwable>> sendDeleteShareGroupOffsetsRequest(Stri | |
| return new SimpleImmutableEntry<>(topLevelException, topicLevelResult); | ||
| } | ||
|
|
||
| void resetOffsets() { | ||
| Map<String, Map<TopicPartition, OffsetAndMetadata>> resetOffsets() { | ||
| Map<String, Map<TopicPartition, OffsetAndMetadata>> result = new HashMap<>(); | ||
|
|
||
| String groupId = opts.options.valueOf(opts.groupOpt); | ||
| try { | ||
| ShareGroupDescription shareGroupDescription = describeShareGroups(List.of(groupId)).get(groupId); | ||
| if (!(GroupState.EMPTY.equals(shareGroupDescription.groupState()) || GroupState.DEAD.equals(shareGroupDescription.groupState()))) { | ||
| CommandLineUtils.printErrorAndExit(String.format("Share group '%s' is not empty.", groupId)); | ||
| } | ||
| resetOffsetsForInactiveGroup(groupId); | ||
| result.put(groupId, resetOffsetsForInactiveGroup(groupId)); | ||
| } catch (InterruptedException ie) { | ||
| throw new RuntimeException(ie); | ||
| } catch (ExecutionException ee) { | ||
| Throwable cause = ee.getCause(); | ||
| if (cause instanceof GroupIdNotFoundException) { | ||
| resetOffsetsForInactiveGroup(groupId); | ||
| result.put(groupId, resetOffsetsForInactiveGroup(groupId)); | ||
| } else if (cause instanceof KafkaException) { | ||
| CommandLineUtils.printErrorAndExit(cause.getMessage()); | ||
| } else { | ||
| throw new RuntimeException(cause); | ||
| } | ||
| } | ||
|
|
||
| return result; | ||
| } | ||
|
|
||
| private void resetOffsetsForInactiveGroup(String groupId) { | ||
| private Map<TopicPartition, OffsetAndMetadata> resetOffsetsForInactiveGroup(String groupId) { | ||
| try { | ||
| Collection<TopicPartition> partitionsToReset = getPartitionsToReset(groupId); | ||
| Map<TopicPartition, OffsetAndMetadata> offsetsToReset = prepareOffsetsToReset(partitionsToReset); | ||
| Map<TopicPartition, OffsetAndMetadata> offsetsToReset = prepareOffsetsToReset(groupId, partitionsToReset); | ||
| boolean dryRun = opts.options.has(opts.dryRunOpt) || !opts.options.has(opts.executeOpt); | ||
| if (!dryRun) { | ||
| adminClient.alterShareGroupOffsets(groupId, | ||
|
|
@@ -418,7 +453,7 @@ private void resetOffsetsForInactiveGroup(String groupId) { | |
| withTimeoutMs(new AlterShareGroupOffsetsOptions()) | ||
| ).all().get(); | ||
| } | ||
| OffsetsUtils.printOffsetsToReset(Map.of(groupId, offsetsToReset)); | ||
| return offsetsToReset; | ||
| } catch (InterruptedException ie) { | ||
| throw new RuntimeException(ie); | ||
| } catch (ExecutionException ee) { | ||
|
|
@@ -448,14 +483,32 @@ private Collection<TopicPartition> getPartitionsToReset(String groupId) throws E | |
| return partitionsToReset; | ||
| } | ||
|
|
||
| private Map<TopicPartition, OffsetAndMetadata> prepareOffsetsToReset(Collection<TopicPartition> partitionsToReset) { | ||
| private Map<TopicPartition, SharePartitionOffsetInfo> getOffsetInfo(String groupId) { | ||
| try { | ||
| return adminClient.listShareGroupOffsets( | ||
| Map.of(groupId, new ListShareGroupOffsetsSpec()), | ||
| withTimeoutMs(new ListShareGroupOffsetsOptions()) | ||
| ).partitionsToOffsetInfo(groupId).get(); | ||
| } catch (InterruptedException | ExecutionException e) { | ||
| throw new RuntimeException(e); | ||
| } | ||
| } | ||
|
|
||
| private Map<TopicPartition, OffsetAndMetadata> prepareOffsetsToReset(String groupId, Collection<TopicPartition> partitionsToReset) { | ||
| offsetsUtils.checkAllTopicPartitionsValid(partitionsToReset); | ||
| if (opts.options.has(opts.resetToEarliestOpt)) { | ||
| if (opts.options.has(opts.resetToOffsetOpt)) { | ||
| return offsetsUtils.resetToOffset(partitionsToReset); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Not related to this pr, but this call sets offset to 0L, if opts.resetToOffsetOpt is empty or null. Is that ok instead of throwing an exception? |
||
| } else if (opts.options.has(opts.resetToEarliestOpt)) { | ||
| return offsetsUtils.resetToEarliest(partitionsToReset); | ||
| } else if (opts.options.has(opts.resetToLatestOpt)) { | ||
| return offsetsUtils.resetToLatest(partitionsToReset); | ||
| } else if (opts.options.has(opts.resetToDatetimeOpt)) { | ||
| return offsetsUtils.resetToDateTime(partitionsToReset); | ||
| } else if (offsetsUtils.resetPlanFromFile().isPresent()) { | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This call resetPlanFromFile and next call resetFromFile seem to do same reading and parsing the entire csv. May be use opts to check for presence? |
||
| return offsetsUtils.resetFromFile(groupId); | ||
| } else if (opts.options.has(opts.resetToCurrentOpt)) { | ||
| Map<TopicPartition, SharePartitionOffsetInfo> currentOffsets = getOffsetInfo(groupId); | ||
| return offsetsUtils.resetToCurrentForShareGroup(partitionsToReset, currentOffsets); | ||
| } | ||
| CommandLineUtils | ||
| .printUsageAndExit(opts.parser, String.format("Option '%s' requires one of the following scenarios: %s", opts.resetOffsetsOpt, opts.allResetOffsetScenarioOpts)); | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is it better to interrupt the currentThread for RunTimeException?
For ExecutionException, getting the instance type for ex if it's of KafkaException(GroupAuthorizationException etc), it could be more visible to the caller ?