Versions Compared

Key

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

Table of Contents

 

Status

Current state:  ["Under Discussion"] Accepted

Discussion thread: hereTBD

JIRA: TBD

Jira
serverASF JIRA
columnskey,summary,type,created,updated,due,assignee,reporter,priority,status,resolution
maximumIssues20
jqlQuerykey = KAFKA-6170 OR key = KAFKA-6150 OR key = KAFKA-6126
serverId5aa69414-a9e9-3523-82ec-879b028fb15b

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

...

Note that the scope of this KIP is only for the user facing public APIs that are needed for adding this client. We are not going to replace the whole StreamsKafkaClient completely within in steps alongside with this KIP.

 

Public Interfaces

...

Code Block
languagejava
import org.apache.kafka.clients.admin.AdminClient;
 
public interface KafkaClientSupplier {
    // other existing APIs

    /**
     * Create a {@link AdminClient} which is used for internal topic management.
     *
     * @param config Supplied by the {@link StreamsConfig} given to the {@link KafkaStreams}
     * @return an instance of {@link AdminClient}
     */
    AdminClient getAdminCLientgetAdminClient(final Map<String, Object> config);
}

 

Proposed Changes

Describe the new thing you want to do in appropriate detail. This may be fairly extensive and have large subsections of its own. Or it may be a few sentences. Use judgement based on the scope of the change.

Compatibility, Deprecation, and Migration Plan

  • What impact (if any) will there be on existing users?
  • If we are changing behavior how will we phase out the older behavior?
  • If we need special migration tools, describe them here.
  • When will we remove the existing behavior?

Rejected Alternatives

The newly added API allows the Streams library to create an instance of AdminClient. We are going to create one instance within each thread, and pass in that instance into the created InternalTopicManager for the leader of the group only. Here are a list of changes we are going to make:

  • Purge repartition data on commit: this is summarized in 
    Jira
    serverASF JIRA
    serverId5aa69414-a9e9-3523-82ec-879b028fb15b
    keyKAFKA-6150
    . The AdminClient's deleteRecords API (adding in KIP-204) will be used upon committing intervals.
  • Create internal topic within InternalTopicManager: we will use the create topic API of AdminClient.
  • We'll also remove the endless-loop checking after the creation within StreamPartitionAssignor (
    Jira
    serverASF JIRA
    serverId5aa69414-a9e9-3523-82ec-879b028fb15b
    keyKAFKA-6126
    ); instead after the rebalance we will let the restoration retry within the main loop if the metadata is not known yet.
  • We'll add an ADMIN_PREFIX (String contant valued "admin.") in the StreamsConfig for admin client usage which will allow certain config parameters for the admin client only, also add the adminClientPrefix function; this is similar to PRODUCER_PREFIX and CONSUMER_PREFIX.
  • Compatibility check: we will remove this functionality and defer to the runtime to detect the returned error code and throw exception.

 

In addition, the following new public APIs will be added as part of this KIP (details of these changes can be found in https://github.com/apache/kafka/pull/4315):

  • Overloaded KafkaStreams constructor to allow overriding the `Time` object.
  • Overloaded AbstractConfig#originalsWithPrefix() to allow specifying whether or not to strip the prefix in the key of the returned map.

 

 

Regarding the changes to replace StreamsKafkaClient with the KafkaAdminClient, the only differences are the configs. Here is a list of configs defined in StreamsConfig that will be used in StreamsKafkaClient:

  • StreamsConfig.CLIENT_ID_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.RETRY_BACKOFF_MS_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.METADATA_MAX_AGE_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.CONNECTIONS_MAX_IDLE_MS_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.RECONNECT_BACKOFF_MS_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.RECONNECT_BACKOFF_MAX_MS_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.SEND_BUFFER_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.RECEIVE_BUFFER_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.REQUEST_TIMEOUT_MS_CONFIG: still used in KafkaAdminClient.
  • StreamsConfig.BOOTSTRAP_SERVERS_CONFIG: still used in KafkaAdminClient.


So none of the defined properties in StreamsConfig need to be deprecated or removed. No Changes here.

 

Compatibility, Deprecation, and Migration Plan

  • Since we are only adding a new function in the public API, it is binary compatible though not source compatible; users are only required to make one-line change and recompile if they customize the KafkaClientSupplier.
  • Note that the admin metrics would be a bit different:
    • "clientID" : [user-value] -> "[clientId]-admin"
    • "type": "kafka-client-metrics" -> "admin-client-metrics"

Rejected Alternatives

NoneIf there are alternative ways of accomplishing the same thing, what were they? The purpose of this section is to motivate why the design is the way it is and not some other way.