You are viewing an old version of this page. View the current version.

Compare with Current View Page History

« Previous Version 9 Next »

Status

Current stateUnder Discussion

Discussion thread: here 

JIRA: here

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

Motivation

Currently, while building a new Topology through the KStreams DSL the processors are automatically named.

The genarated names are prefixed depending of the operation (i.e KSTREAM-SOURCE, KSTREAM-FILTER, KSTREAM-MAP, etc).  Moreover, they are added with a suffix like "-0000000000" to guarantee their uniqueness.

To debug/understand a topology it is possible to display the processor lineage with the method Topology#describe(). However, a complex topology with dozens of operations can be hard to understand if the processor names are not relevant.

So allowing users to be able to set more meaningful names can help to resolve complexity of some developed topologies. For example, a processor name could describe the business rule performed by a map() operation.

In addition, the generated names have a few disadvantages to guarantee topology compatibilities. In fact, adding a new operator, using a third-library doing some optimization to remove some operators or upgrading to a new KafkaStreams version with internal API changes may changed suffix indexing for a large amount of the processor names. This will in turn change the internal state store names, as well as internal topic names as well.

Public Interfaces

First, we propose to add one new interface Named that can be used to customize stateless operations as well as stateful ones. The objective to create a new class is to keep consistent with the overall API design.

package org.apache.kafka.streams.kstream;

/**
 * Default interface which can be used to personalized the named of both {@link KStream} and {@link KTable} operations.
 */
public interface Named {

    default String name() {
        return null;
    }

    /**
     * Creates an instance with the specified name.
     *
     * @param name  the name to be used.
     * @return a new {@link Named} instance .
     */
    static Named with(final String name) {
        return new Named() {
            @Override
            public String name() {
                return name;
            }
        };
    }
}


The Named interface will be extends by all existing action interfaces :

  • Aggregator
  • ForEachAction
  • KeyValueMapper
  • Predicate
  • Reducer
  • ValueJoiner
  • ValueMapper
  • ValueMapperWithKey

The name() method should be defined as a default method to not break compatibility with previous APIs. In addition, the default method allow us to still used lambda functions introduced with java 8.

Each interface will be enriched with a new static method named to keep a simple and elegant API for developers.

Example for the interface  Predicate



public interface Predicate<K, V> extends Named {

    /**
     * Test if the record with the given key and value satisfies the predicate.
     *
     * @param key   the key of the record
     * @param value the value of the record
     * @return {@code true} if the {@link KeyValue} pair satisfies the predicate&mdash;{@code false} otherwise
     */
    boolean test(final K key, final V value);

    /**
     * Creates a new operation with the specified name.
     *
     * @param name          the name to be used for generated processors applied to this stream.
     * @param predicate     the predicate.
     * @param <K>           the type of keys
     * @param <V>           the type of values
     * @return a new {@link Predicate} instance.
     */
    static <K, V> Predicate<K, V> named(final String name, final Predicate<K, V> predicate) {
        return new Predicate<K, V>() {
            @Override
            public boolean test(K key, V value) {
                return predicate.test(key, value);
            }

            @Override
            public String name() {
                return name;
            }
        };
    }
}



Stateless operations

Stateless operations like foreach(), map(), flatMap(), mapValues(), filter(),filterNot,  are translated into a single processor. The action interfaces depicted above will be used to directly change the processor name (i.e ForEachAction, ValueMapper, ValueMapperWithKey KeyValueMapper).

The method branch() results in multiple streams that can be described with the used of the interface Predicate. However the name of the first translated processor will not be customizable because this would require to overload method with a one accepting a Named instance in argument.  This seems to be OK as the processor name would not determine any related topic / store names.

Statefull operations

Stateful operations like join(), leftjoin(), outerJoin() are translated to multiple processors. The given name should be used to describe the first processor and as a prefix for all subsequent processors. The interface ValueJoiner will be used to customize the processor names.


We propose to overload each of those methods to accept a Named agument :

  • KStream/KTable#transform()

  • KStream/KTable#transformValues()

  • KStream#process()

Source and Sink operations

Then, for those  methods we propose to add a new method withProcessorName() to classes Consumed, Produced and Printed in order to minimize the number of overloaded methods.

This will allow developers to specify a processor name for operations: 

  • <K, V> KTable<K, V> table(final String topic, final Consumed<K, V> consumed)
  • <K, V> KStream<K, V> stream(final String topic, final Consumed<K, V> consumed)
  • void print(final Printed<K, V> printed)
  • void to(final String topic, final Produced<K, V> produced)
  • void to(final TopicNameExtractor<K, V> topicExtractor, final Produced<K, V> produced)

Example for the Produced class : 

/**
 * Create an instance of {@link Produced} with provided processor name.
 *
 * @param processorName the processor name to be used. If {@code null} a default processor name will be generated
 * @param <K>         key type
 * @param <V>         value type
 * @return a new instance of {@link Produced}
 */
public static <K, V> Produced<K, V> with(final String processorName) {
    return new Produced<>(null, null, null, null, processorName);
}

/**
 * Configure the instance of {@link Produced} with a key {@link Serde}.
 *
 * @param processorName the processor name to be used. If {@code null} a default processor name will be generated
 * @return this
 */
public Produced<K, V> withProcessorName(final String processorName) {
    this.processorName = processorName;
    return this;
}


Proposed Changes

  • Implement the new interface Named and updateall action interfaces
  • Overload methods process(), transform() and transformValues() for classes KStreams and KTables.
  • Update the Consumed, Produced and Printed classes to be able to set a processor name.
  • The processor names specified by developer will be used in place of the static processor prefix. Statics prefixes will still be used if no custom processor name are specified.
  • Processor names should follow the same restrictions as the topic names. So legal characters are [a-zA-Z0-9._-] and the maximum length us 249.


Below is an application example : 

final StreamsBuilder builder = new StreamsBuilder();


builder.stream("topic-input", Consumed.with("STREAM-FROM-TOPIC-INPUT")
        .filter(Predicate.named("FILTER-NULL-VALUE", (k, v) -> true ))
        .map(KeyValueMapper.named("MAP-TO-UPPERCASE", (k, v) -> KeyValue.pair(k, v.toUpperCase()))
        .to("topic-output", Produced.with("TO-OUTPUT-TOPIC"));

System.out.println(builder.build().describe());
---- (output)----
Topologies:
   Sub-topology: 0
    Source: STREAM-FROM-TOPIC-INPUT (topics: [topic-input])
      --> FILTER-NULL-VALUE
    Processor: FILTER-NULL-VALUE (stores: [])
      --> MAP-TO-UPPERCASE
      <-- STREAM-FROM-TOPIC-INPUT
    Processor: MAP-TO-UPPERCASE (stores: [])
      --> TO-OUTPUT-TOPIC
      <-- FILTER-NULL-VALUE
    Sink: TO-OUTPUT-TOPIC (topic: topic-output)
      <-- MAP-TO-UPPERCASE

Compatibility, Deprecation, and Migration Plan

No compatibility issues foreseen.

Rejected Alternatives

  1. The first proposition was to overload all stateless methods to accept an instance of Described class. However this solution was resulting in modiying a large percentage of the existing KStream and KTable methods.
  2. The second proposition was to add new methods KStreams#as(Described) and KTable#as(Described) while Described class would be used to customized the named of operation defined previously in the stream. However not only this new method was not conservative with the existing APIs but it also introduce some complexities for methods returning Void.


  • No labels