Status

Current state: Adopted

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

JIRA: KAFKA-12849

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

Motivation

This KIP could be seen as an spin-off of KIP-740: Clean up public API in 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.

Within this KIP discussion, it was also pointed out that ThreadMetadata shares the same problems as TaskMetadata. Hence, this KIP proposes, as well, to convert ThreadMetadata into an interface and keep the implementation as an internal class. The reasoning behind this is the same as with the previous class, ThreadMetadata was never meant to be instantiated by end users, but was rather meant as an API.

Also it is important to notice that the goal of this KIP needs to still be consistent with that of KIP-740, which was to not expose TaskID as a String in the public APIs.

For reasons described in further detail under "Public Interfaces" this KIP should be part of release 3.0.0, so we can keep the APIs consistent without the need to wait for a full deprecation cycle.

Public Interfaces

Create a new Interface named TaskMetadata under package org.apache.kafka.streams, which will contain all currently non deprecated methods:

org.apache.kafka.streams.TaskMetadata
/**
 * Represents the state of a single task running within a {@link KafkaStreams} application.
 */
public interface TaskMetadata {


    /**
     * @return the basic task metadata such as subtopology and partition id
     */
    TaskId taskId();

    /**
     * This function will return a set of the current TopicPartitions
     */
    Set<TopicPartition> topicPartitions();

    /**
     * This function will return a map of TopicPartitions and the highest committed offset seen so far
     */
    Map<TopicPartition, Long> committedOffsets();

    /**
     * This function will return a map of TopicPartitions and the highest offset seen so far in the Topic
     */
    Map<TopicPartition, Long> endOffsets();

    /**
     * This function will return the time task idling started, if the task is not currently idling it will return empty
     */
    Optional<Long> timeCurrentIdlingStarted();

	/**
     * Compares the specified object with this TaskMetadata. Returns {@code true} if and only if the specified object is
     * also a TaskMetadata and both {@code taskId()} and {@code topicPartitions()} are equal.
     */
    boolean equals(final 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(taskId(), topicPartitions());
     * }
     * </pre>
     */
    int hashCode();
}


Additionally, TaskMetadata under org.apache.kafka.streams.processor, will be deprecated:

org.apache.kafka.streams.processor.TaskMetadata
/**
 * Represents the state of a single task running within a {@link KafkaStreams} application.
 * @deprecated since 3.0, use {@link org.apache.kafka.streams.TaskMetadata} instead
 */
@Deprecated
public class TaskMetadata {
...
}

Furthermore, the method getTaskId newly introduced in KIP-740 (but not yet released) will be removed, as it won't be needed any more. For this reason, it makes sense to include this KIP in release 3.0.0 so we can avoid a full cycle of deprecation for this newly introduced method.

org.apache.kafka.streams.processor.TaskMetadata
// This method will be removed in this KIP
/**
 * @return the basic task metadata such as subtopology and partition id
*/
public TaskId getTaskId() {
   return taskId;
}


ThreadMetadata class under org.apache.kafka.streams.processor will be deprecated:

org.apache.kafka.streams.processor.ThreadMetadata
/**
 * Represents the state of a single thread running within a {@link KafkaStreams} application.
 * @deprecated since 3.0 use {@link org.apache.kafka.streams.ThreadMetadata} instead
 */
@Deprecated
public class ThreadMetadata {
...
}


A new interface named ThreadMetadata will be introduced under org.apache.kafka.streams:

org.apache.kafka.streams.ThreadMetadata
/**
 * Represents the state of a single thread running within a {@link KafkaStreams} application.
 */
public interface ThreadMetadata {


    /**
     * @return the state of the Thread
     */
    String threadState();

    /**
     * @return the name of the Thread
     */
    String threadName();

    /**
     * This function will return the set of the {@link TaskMetadata} for the current active tasks
     */
    Set<TaskMetadata> activeTasks();

    /**
     * This function will return the set of the {@link TaskMetadata} for the current standby tasks
     */
    Set<TaskMetadata> standbyTasks();

    /**
     * @return the consumer Client Id
     */
    String consumerClientId();

    /**
     * @return the restore consumer Client Id
     */
    String restoreConsumerClientId();

    /**
     * This function will return the set of Client Ids for the producers
     */
    Set<String> producerClientIds();

    /**
     * @return the admin Client Id
     */
    String adminClientId();

    /**
     * Compares the specified object with this ThreadMetadata. Returns {@code true} if and only if the specified object is
     * also a TaskMetadata and both {@code threadName()} are equal, {@code threadState()} are equal, {@code activeTasks()} contain the same
     * elements, {@code standbyTasks()} contain the same elements, {@code mainConsumerClientId()} are equal, {@code restoreConsumerClientId()}
     * are equal, {@code producerClientIds()} are equal, {@code producerClientIds} contain the same elements, and {@code adminClientId()} are equal.
     */
    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(
     *             threadName,
     *             threadState,
     *             activeTasks,
     *             standbyTasks,
     *             mainConsumerClientId,
     *             restoreConsumerClientId,
     *             producerClientIds,
     *             adminClientId);
     * </pre>
     */
    int hashCode();
}


Additionally, KafkaStreams class needs to adapted to return the new ThreadMetadata interface, and have the old method (returning the old class) deprecated as follows:


org.apache.kafka.streams.KafkaStreams
/**
 * 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;
}




UPDATE

During the vote thread discussion it was requested if it would be possible to align this KIP with Unable to render Jira issues macro, execution error. . 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 under org.apache.kafka.streams with an internal implementation:


org.apache.kafka.streams.StreamsMetadata
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:


org.apache.kafka.streams.state.StreamsMetadata
/**
 * 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:


org.apache.kafka.streams.KafkaStreams
/**
     * 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 follow a similar pattern. A new class StreamsMetadataImpl will be created under org.apache.kafka.streams.state.internals, which will implement the newly created interface org.apache.kafka.streams.StreamsMetadata. 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 new interfaces, 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 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.

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 Unable to render Jira issues macro, execution error. 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.


  • No labels