Skip to content
Open
Show file tree
Hide file tree
Changes from all 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
27 changes: 27 additions & 0 deletions cmd/topicctl/subcmd/delete.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ func init() {
addSharedFlags(deleteCmd, &deleteConfig.shared)
deleteCmd.AddCommand(
deleteACLCmd(),
deleteGroupCmd(),
)
RootCmd.AddCommand(deleteCmd)
}
Expand Down Expand Up @@ -150,3 +151,29 @@ $ topicctl delete acls --resource-type topic --resource-pattern-type literal --r
cmd.MarkFlagRequired("resource-type")
return cmd
}

func deleteGroupCmd() *cobra.Command {
cmd := &cobra.Command{
Use: "group [group]",
Short: "Delete a given consumer group. Ensure the group is not active before deleting.",
Args: cobra.ExactArgs(1),
Example: `Delete group my-group`,
RunE: func(cmd *cobra.Command, args []string) error {
ctx := context.Background()
sess := session.Must(session.NewSession())

adminClient, err := deleteConfig.shared.getAdminClient(ctx, sess, deleteConfig.dryRun)
if err != nil {
return err
}
defer adminClient.Close()

group := args[0]
cliRunner := cli.NewCLIRunner(adminClient, log.Infof, !noSpinner)
return cliRunner.DeleteGroup(ctx, group)
},
}

return cmd

}
13 changes: 13 additions & 0 deletions pkg/cli/cli.go
Original file line number Diff line number Diff line change
Expand Up @@ -648,6 +648,19 @@ func (c *CLIRunner) ResetOffsets(
return nil
}

// DeleteGroup deletes a single consumer group.
func (c *CLIRunner) DeleteGroup(ctx context.Context, groupID string) error {
c.startSpinner()
err := groups.Delete(ctx, c.adminClient.GetConnector(), groupID)
c.stopSpinner()
if err != nil {
return err
}

c.printer("Success")
return nil
}

// Tail prints out a stream of the latest messages in a topic.
func (c *CLIRunner) Tail(
ctx context.Context,
Expand Down
25 changes: 25 additions & 0 deletions pkg/groups/groups.go
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,31 @@ func ResetOffsets(
)
}

// Delete deletes a consumer group based on its groupId.
func Delete(ctx context.Context, connector *admin.Connector, groupID string) error {
describeGroupsRequest := kafka.DescribeGroupsRequest{
GroupIDs: []string{groupID},
}
describeGroupsResponse, err := connector.KafkaClient.DescribeGroups(ctx, &describeGroupsRequest)
if err != nil {
return err
}

if len(describeGroupsResponse.Groups) != 1 {
return errors.New("Unexpected response length from describeGroups")
}

if describeGroupsResponse.Groups[0].GroupState == "Dead" {
return errors.New("Group state is dead; check that group ID is valid")
}

req := kafka.DeleteGroupsRequest{
GroupIDs: []string{groupID},
}
_, err = connector.KafkaClient.DeleteGroups(ctx, &req)
return err
}

// GetEarliestorLatestOffset gets earliest/latest offset for a given topic partition for resetting offsets of consumer group
func GetEarliestOrLatestOffset(
ctx context.Context,
Expand Down