Versions Compared

Key

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

Table of Contents

Status

Current state: Under DiscussionAdopted

Discussion thread: https://lists.apache.org/x/thread.html/r1d20fb6dbd6b01bb84cbb17e992f4d08308980dfc5f2e0a68d674413@%3Cdev.kafka.apache.org%3E

...

This KIP could be seen as an spin-off of KIP-740: Clean up public API in TaskId and fix TaskMetadata#taskId(). During the one of the Pull Requests that were created to implement the aforementioned KIP, it became apparent that TaskMetadata was never a class that the user would ever need to instantiate, but rather an API for exposing metadata. For this reason, it would make sense to separate the public interface from the internal implementation, hence this proposal to create the TaskMetadata interface. See https://github.com/apache/kafka/pull/10755#discussion_r639338584 for further context on this topic.

...

Code Block
languagejava
titleorg.apache.kafka.streams.KafkaStreams.java
linenumberstrue
	/**
     * Returns runtime information about the local threads of this {@link KafkaStreams} instance.
     *
     * @return the set of {@link org.apache.kafka.streams.processor.ThreadMetadata}.
     * @deprecated since 3.0 use {@link #threadsMetadata()}
     */
    @Deprecated
    public Set<org.apache.kafka.streams.processor.ThreadMetadata> localThreadsMetadata() {
        return threadsMetadata().stream().map(threadMetadata -> new org.apache.kafka.streams.processor.ThreadMetadata(
                threadMetadata.threadName(),
                threadMetadata.threadState(),
                threadMetadata.consumerClientId(),
                threadMetadata.restoreConsumerClientId(),
                threadMetadata.producerClientIds(),
                threadMetadata.adminClientId(),
                threadMetadata.activeTasks().stream().map(taskMetadata -> new TaskMetadata(
                        taskMetadata.taskId().toString(),
                        taskMetadata.topicPartitions(),
                        taskMetadata.committedOffsets(),
                        taskMetadata.endOffsets(),
                        taskMetadata.timeCurrentIdlingStarted())
                ).collect(Collectors.toSet()),
                threadMetadata.standbyTasks().stream().map(taskMetadata -> new TaskMetadata(
                        taskMetadata.taskId().toString(),
                        taskMetadata.topicPartitions(),
                        taskMetadata.committedOffsets(),
                        taskMetadata.endOffsets(),
                        taskMetadata.timeCurrentIdlingStarted())
                ).collect(Collectors.toSet())))
                .collect(Collectors.toSet());
    }
 
    /**
     * Returns runtime information about the local threads of this {@link KafkaStreams} instance.
     *
     * @return the set of {@link ThreadMetadata}.
     **/
    public Set<ThreadMetadata> threadsMetadata() {
        final Set<ThreadMetadata> threadMetadata = new HashSet<>();
        processStreamThread(thread -> {
            synchronized (thread.getStateLock()) {
                if (thread.state() != StreamThread.State.DEAD) {
                    threadMetadata.add(thread.getThreadMetadata());
                }
            }
        });
        return threadMetadata;
    }

Proposed Changes




UPDATE

During the vote thread discussion it was requested if it would be possible to align this KIP with

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12370
. This seems doable only with one of the changes that shows some concerns. Discussion: https://lists.apache.org/x/thread.html/rfaaa8c40903eca9ba2c8b406c486e6beee3fa916942c28446358c9b1@%3Cdev.kafka.apache.org%3E

The first one is converting StreamsMetadata into an interface By introducing a new TaskMetadata interface on a different package, we are able to keep the same name (no need to come up with some forced name), and also, we are freed from carrying over changes like the one introduced in https://github.com/apache/kafka/pull/10755 where a method getTaskId was introduced (going against Kafka naming convention) because taskId was already taken and needed to be deprecated.
Together with this change, a new class TaskMetadataImpl will be created under org.apache.kafka.streams .processor.internals, which will implement the aforementioned interface with the implementation present on the with an internal implementation:


Code Block
languagejava
titleorg.apache.kafka.streams.StreamsMetadata
linenumberstrue
public interface StreamsMetadata {

    /**
     * The value of {@link StreamsConfig#APPLICATION_SERVER_CONFIG} configured for the streams
     * instance, which is typically host/port
     *
     * @return {@link HostInfo} corresponding to the streams instance
     */
    HostInfo hostInfo();

    /**
     * State stores owned by the instance as an active replica
     *
     * @return set of active state store names
     */
    Set<String> stateStoreNames();

    /**
     * Topic partitions consumed by the instance as an active replica
     *
     * @return set of active topic partitions
     */
    Set<TopicPartition> topicPartitions();

    /**
     * (Source) Topic partitions for which the instance acts as standby.
     *
     * @return set of standby topic partitions
     */
    Set<TopicPartition> standbyTopicPartitions();

    /**
     * State stores owned by the instance as a standby replica
     *
     * @return set of standby state store names
     */
    Set<String> standbyStateStoreNames();

    /**
     * This method is equivalent to call {@code StreamsMetadata.hostInfo().host();}
     */
    String host();

    /**
     * This method is equivalent to call {@code StreamsMetadata.hostInfo().port();}
     */
    int port();

    /**
     * Compares the specified object with this StreamsMetadata. Returns {@code true} if and only if the specified object is
     * also a StreamsMetadata and for both {@code hostInfo()} are equal, and {@code stateStoreNames()}, {@code topicPartitions()},
     * {@code standbyStateStoreNames()}, and {@code standbyTopicPartitions()} contain the same elements.
     */
    boolean equals(Object o);

    /**
     * Returns the hash code value for this TaskMetadata. The hash code of a list is defined to be the result of the following calculation:
     * <pre>
     * {@code
     * Objects.hash(hostInfo(), stateStoreNames(), topicPartitions(), standbyStateStoreNames(), standbyTopicPartitions());
     * }
     */
    int hashCode();

}


Along with this change, the org.apache.kafka.streams.state.StreamsMetadata is deprecated:


Code Block
languagejava
titleorg.apache.kafka.streams.state.StreamsMetadata
linenumberstrue
/**
 * Represents the state of an instance (process) in a {@link KafkaStreams} application.
 * It contains the user supplied {@link HostInfo} that can be used by developers to build
 * APIs and services to connect to other instances, the Set of state stores available on
 * the instance and the Set of {@link TopicPartition}s available on the instance.
 * NOTE: This is a point in time view. It may change when rebalances happen.
 * @deprecated since 3.0.0 use {@link org.apache.kafka.streams.StreamsMetadata}
 */
@Deprecated
public class StreamsMetadata {
...
}


For this change, KafkaSTreams class needs to be adapted to return the new StreamsMetadata interface instead, while at the same time deprecate the old method:


Code Block
languagejava
titleorg.apache.kafka.streams.KafkaStreams
linenumberstrue
/**
     * Find all currently running {@code KafkaStreams} instances (potentially remotely) that use the same
     * {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all instances that belong to
     * the same Kafka Streams application) and return {@link StreamsMetadata} for each discovered instance.
     * <p>
     * Note: this is a point in time view and it may change due to partition reassignment.
     *
     * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application
     * @deprecated since 3.0.0 use {@link KafkaStreams#allRunningMetadata}
     */
    @Deprecated
    public Collection<org.apache.kafka.streams.state.StreamsMetadata> allMetadata() {
        validateIsRunningOrRebalancing();
        return streamsMetadataState.getAllMetadata().stream().map(streamsMetadata ->
                new org.apache.kafka.streams.state.StreamsMetadata(streamsMetadata.hostInfo(),
                        streamsMetadata.stateStoreNames(),
                        streamsMetadata.topicPartitions(),
                        streamsMetadata.standbyStateStoreNames(),
                        streamsMetadata.standbyTopicPartitions()))
                .collect(Collectors.toSet());
    }

    /**
     * Find all currently running {@code KafkaStreams} instances (potentially remotely) that use the same
     * {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all instances that belong to
     * the same Kafka Streams application) and return {@link StreamsMetadata} for each discovered instance.
     * <p>
     * Note: this is a point in time view and it may change due to partition reassignment.
     *
     * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances of this application
     */
    public Collection<StreamsMetadata> allRunningMetadata() {
        validateIsRunningOrRebalancing();
        return streamsMetadataState.getAllMetadata();
    }

    /**
     * Find all currently running {@code KafkaStreams} instances (potentially remotely) that
     * <ul>
     *   <li>use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all
     *       instances that belong to the same Kafka Streams application)</li>
     *   <li>and that contain a {@link StateStore} with the given {@code storeName}</li>
     * </ul>
     * and return {@link StreamsMetadata} for each discovered instance.
     * <p>
     * Note: this is a point in time view and it may change due to partition reassignment.
     *
     * @param storeName the {@code storeName} to find metadata for
     * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances with the provide {@code storeName} of
     * this application
     * @deprecated since 3.0.0 use {@link KafkaStreams#allMetadataForGivenStore} instead
     */
    @Deprecated
    public Collection<org.apache.kafka.streams.state.StreamsMetadata> allMetadataForStore(final String storeName) {
        validateIsRunningOrRebalancing();
        return streamsMetadataState.getAllMetadataForStore(storeName).stream().map(streamsMetadata ->
                new org.apache.kafka.streams.state.StreamsMetadata(streamsMetadata.hostInfo(),
                        streamsMetadata.stateStoreNames(),
                        streamsMetadata.topicPartitions(),
                        streamsMetadata.standbyStateStoreNames(),
                        streamsMetadata.standbyTopicPartitions()))
                .collect(Collectors.toSet());
    }

    /**
     * Find all currently running {@code KafkaStreams} instances (potentially remotely) that
     * <ul>
     *   <li>use the same {@link StreamsConfig#APPLICATION_ID_CONFIG application ID} as this instance (i.e., all
     *       instances that belong to the same Kafka Streams application)</li>
     *   <li>and that contain a {@link StateStore} with the given {@code storeName}</li>
     * </ul>
     * and return {@link StreamsMetadata} for each discovered instance.
     * <p>
     * Note: this is a point in time view and it may change due to partition reassignment.
     *
     * @param storeName the {@code storeName} to find metadata for
     * @return {@link StreamsMetadata} for each {@code KafkaStreams} instances with the provide {@code storeName} of
     * this application
     */
    public Collection<StreamsMetadata> allMetadataForGivenStore(final String storeName) {
        validateIsRunningOrRebalancing();
        return streamsMetadataState.getAllMetadataForStore(storeName);
    }


Proposed Changes

By introducing a new TaskMetadata interface on a different package, we are able to keep the same name (no need to come up with some forced name), and also, we are freed from carrying over changes like the one introduced in https://github.com/apache/kafka/pull/10755 where a method getTaskId was introduced (going against Kafka naming convention) because taskId was already taken and needed to be deprecated.
Together with this change, a new class TaskMetadataImpl will be created under org.apache.kafka.streams.processor.internals, which will implement the aforementioned interface with the implementation present on the org.apache.kafka.streams.processor.TaskMetadata. The rest of the Kafka code base using the newly deprecated TaskMetadata will be migrated to use the new classes instead.

The changes related to ThreadMetadata follow a similar pattern. A new class ThreadMetadataImpl will be created under org.apache.kafka.streams.processor.internals, which will implement the newly created interface org.apache.kafka.streams.ThreadMetadata. The rest of the Kafka code base will be migrated to use the newly created classes.

The changes related to StreamsMetadata processor.TaskMetadata. The rest of the Kafka code base using the newly deprecated TaskMetadata will be migrated to use the new classes instead.
The changes related to ThreadMetadata follow a similar pattern. A new class ThreadMetadataImpl StreamsMetadataImpl will be created under org.apache.kafka.streams.processorstate.internals, which will implement the newly created interface org.apache.kafka.streams.ThreadMetadata. The rest of the Kafka code base will be migrated to use the newly created classesStreamsMetadata. The rest of the Kafka code base will be migrated to use the newly created classes.

Unsure change, KafkaStreams would return a Set of StreamsMetadata in threadsMetadata, instead of a Set of ThreadMetada.


Also, by introducing new interfaces altogether, we ease the migration of any possible current user of the newly deprecated classes, offering a smooth transition to the new APIs.

Compatibility, Deprecation, and Migration Plan

As we are creating a new interfaceinterfaces, these changes are 100% compatible. Users who currently use org.apache.kafka.streams.processor.TaskMetadata, should migrate to org.apache.kafka.streams.TaskMetadata.

Same goes for users who currently use org.apache.kafka.streams.processor.ThreadMetadata and org.apache.kafka.streams.state.StreamsMetadta, they should migrate to org.apache.kafka.streams.ThreadMetadata and org.apache.kafka.streams.StreamsMetadata respectively.

Users using KafkaStreams#localThreadsMetadata should migrate to KafkaStreams#threadsMetadata.

...

Classes org.apache.kafka.streams.processor.TaskMetadata and .TaskMetadata and org.apache.kafka.streams.processor.ThreadMetadata will be deprecated and removed in future releases.

Rejected Alternatives

An alternative that was rejected, was to directly transform org.apache.kafka.streams.processor.ThreadMetadata will be deprecated and removed in future releases.

Rejected Alternatives

An alternative that was rejected, was to directly transform org.apache.kafka.streams.TaskMetadata into an interface with this change. This was seen as a breaking change as the current constructor won't be present any more in this alternative, making possible users of this fail to compile.

...

TaskMetadata into an interface with this change. This was seen as a breaking change as the current constructor won't be present any more in this alternative, making possible users of this fail to compile.

Additionally, this change forced us to keep the method `getTaskId` for a while, until we could deprecate and remove the current `taskId` method.


The inclusion of more complex changes mentioned in

Jira
serverASF JIRA
serverId5aa69414-a9e9-3523-82ec-879b028fb15b
keyKAFKA-12370
like changing the return type of localThreadsMetadata to be a Set of StreamsMetadta (instead of a Set of ThreadsMetadta) was rejected as the idea came late into the process and the community wouldn't have enough time to give their opinion on this more significant change.