Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.

Table of Contents

 

Status

Current state:   "Under DiscussionAccepted"

Discussion thread: here

JIRA: here 

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-6058

Released: 2.0.0


Please keep the discussion on the mailing list rather than commenting on the wiki (wiki discussions get unwieldy fast).

...

Code Block
languagejava
public abstract class AdminClient implements AutoCloseable {
 	
	public abstract DescribeGroupsResultDescribeConsumerGroupsResult describeGroupsdescribeConsumerGroups(Collection<String> groupIds, DescribeGroupsOptions DescribeConsumerGroupsOptions options);
 
	public DescribeGroupsResultDescribeConsumerGroupsResult describeGroupsdescribeConsumerGroups(Collection<String> groupIds) {
    	return describeGroupsdescribeConsumerGroups(groupIds, new DescribeGroupsOptionsDescribeConsumerGroupsOptions());
	}
 
    public abstract ListGroupsResultListConsumerGroupsResult listGroupslistConsumerGroups(ListGroupsOptionsListConsumerGroupsOptions options);

    public ListGroupsResultListConsumerGroupsResult listGroupslistConsumerGroups() {
        return listGroupslistConsumerGroups(new ListGroupsOptionsListConsumerGroupsOptions());
    }

    public abstract ListGroupsResultListConsumerGroupOffsetsResult listConsumerGroups(ListGroupsOptionslistConsumerGroupOffsets(String groupId, ListConsumerGroupsOptions options); //filtering groups by ConsumerProtocol.PROTOCOL_TYPE
 
    

    public ListGroupsResultListConsumerGroupOffsetsResult listConsumerGroupslistConsumerGroupOffsets(String groupId) {
        return listGroupslistConsumerGroupOffsets(new ListGroupsOptionsListConsumerGroupOffsetsOptions());
    }
     
    public abstract ListGroupOffsetsResultDeleteConsumerGroupsResult listGroupOffsetsdeleteConsumerGroups(StringCollection<String> groupIdgroupIds, ListGroupsOptionsDeleteConsumerGroupsOptions options);
 
    public ListGroupOffsetsResultDeleteConsumerGroupsResult listGroupOffsetsdeleteConsumerGroups(StringCollection<String> groupIdgroupIds) {
        return listGroupOffsetsdeleteConsumerGroups(groupIds, new ListGroupOffsetsOptionsDeleteConsumerGroupsOptions());
    }
}
 
public class DescribeGroupOptionsDescribeConsumerGroupOptions extends AbstractOptions<DescribeGroupOptions>AbstractOptions<DescribeConsumerGroupOptions> {
}
 
public class DescribeGroupResultDescribeConsumerGroupResult {
    private final Map<StringMap<TopicPartition, KafkaFuture<GroupDescription>>KafkaFuture<ConsumerGroupDescription>> futures;
...all();
}
 
public class GroupDescriptionConsumerGroupDescription {
    private final String groupId;
    private final String protocolType;
    private final List<MemberDescription> members/**
     * The id of the consumer group.
     */
    public String groupId();

    /**
     * If consumer group is simple or not.
     */
    public boolean isSimpleConsumerGroup();

    /**
     * A list of the members of the consumer group.
     */
    public Collection<MemberDescription> members();

    /**
     * The consumer group partition assignor.
     */
    public String partitionAssignor();

    /**
     * The consumer group state, or UNKNOWN if the state is too new for us to parse.
     */
    public ConsumerGroupState state();

    /**
     * The consumer group coordinator, or null if the coordinator is not known.
     */
    public Node coordinator();

}


public enum ConsumerGroupState {
    UNKNOWN,
    PREPARING_REBALANCE,
    COMPLETING_REBALANCE,
    STABLE,
    DEAD,
    EMPTY;
}
 
public class MemberDescription {

    /**
    private final * The consumer id of the group member.
     */
    public String consumerId();

    /**
     * The client id of the group member.
    private final */
    public String clientId();

    private final/**
     * The host where the group member is running.
     */
    public String host();

    /**
     * The assignment of the group member.
     private final*/
    public MemberAssignment assignment();
}
 
public class MemberAssignment {
    private final List<TopicPartition> assignmenttopicPartitions;
}

public class ListGroupsOptionsListConsumerGroupsOptions extends AbstractOptions<ListGroupsOptions>AbstractOptions<ListConsumerGroupsOptions> {
}

public class ListGroupsResultListConsumerGroupsResult {
    finalpublic KafkaFuture<Map<String, GroupListing>> future;

    //...
}KafkaFuture<Collection<ConsumerGroupListing>> all();
    public KafkaFuture<Collection<ConsumerGroupListing>> valid();
	public KafkaFuture<Collection<Throwable>> errors();
} 
 
public class GroupListingConsumerGroupListing {
    private final String namegroupId;
    private final Stringboolean protocolTypeisSimpleConsumerGroup;
}
 
public class ListGroupOffsetsOptionsListConsumerGroupOffsetsOptions extends AbstractOptions<ListGroupsOptions>AbstractOptions<ListConsumerGroupsOptions> {
}
 
public class ListGroupOffsetsResultListConsumerGroupOffsetsResult {
    final KafkaFuture<Map<TopicPartition, OffsetAndMetadata>> futureall();
}
 
public class DeleteConsumerGroupOptions  //...extends AbstractOptions<DeleteConsumerGroupOptions> {
}
 
public class DeleteConsumerGroupsResult {
    Map<TopicPartition, KafkaFuture<Void>> all();
}

This API returns a future object whose result will be available within RequestTimeoutMs, which is configured when user constructs the AdminClient.

Proposed Changes

  1. Add `#describeGroups(Collection<String> groupIds, DescribeGroupsOptions options)` and `#describeGroups(Collection<String> groupIds)` to `AdminClient` APIAdd `#describeConsumerGroups(Collection<String> groupIds, DescribeGroupsOptions DescribeConsumerGroupsOptions options)` and `#describeConsumerGroups(Collection<String> groupIds)` to `AdminClient` API
  2. Add `#listGroups`#listConsumerGroups(ListGroupsOptions ListConsumerGroupsOptions options)` and `#listGroups`#listConsumerGroups()` to `AdminClient` API 
  3. Add `#listGroupOffsets`#listConsumerGroupOffsets(String groupId, ListGroupOffsetsOptions ListConsumerGroupOffsetsOptions options)` and `#listGroupOffsets`#listConsumerGroupOffsets(String groupId)` to `AdminClient` API and
  4. Implement it on `KafkaAdminClient`.
  5. Solve 
    Jira
    serverASF JIRA
    serverId5aa69414-a9e9-3523-82ec-879b028fb15b
    keyKAFKA-6058
  6. Add `#deleteConsumerGroups(Collection<String> groupIds, ConsumerGroupsOptions options)` and `#deleteConsumerGroups(Collection<String> groupIds)` to `AdminClient` API based on KIP-229: DeleteGroups API.

Compatibility, Deprecation, and Migration Plan

  • There won't be any impact on existing users.
  • There won't be any change of current behavior.
  • No migration tool required.

Rejected Alternatives

  • We are moving one more functionality from `core`'s `AdminClient` API to `clients`'s `AdminClient` API. This is aim to remove dependencies to `core` module.
  • Aggregating both Consumer Groups and Connect Groups in the same API would increase complexity adding generic MemberAssignment to match both groups. This KIP is only including Consumer Groups and will leave Connect Groups to be handled by another KIP. https://github.com/apache/kafka/pull/4454#issuecomment-360553277 

Future Work

  • Streams Resetter Tool will be remove it's dependency to `core` module for `KafkaAdminClient`.

References:

Pull Request: https://github.com/apache/kafka/pull/4454