You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by GitBox <gi...@apache.org> on 2020/07/30 22:30:44 UTC

[GitHub] [kafka] inponomarev opened a new pull request #9107: KAFKA-5488: KIP-418 implementation

inponomarev opened a new pull request #9107:
URL: https://github.com/apache/kafka/pull/9107


   
   ### Committer Checklist (excluded from commit message)
   - [ ] Verify design and implementation 
   - [ ] Verify test coverage and CI build status
   - [ ] Verify documentation (including upgrade notes)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax merged pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax merged pull request #9107:
URL: https://github.com/apache/kafka/pull/9107


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547591511



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead

Review comment:
       `you can apply [multiple] {#filter} [operators], one for each predicate, instead`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Then we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   is to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. It's best to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further debates :-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-752241886


   To make the build pass, for now, it should be sufficient to just deprecate the method via `@nowarn("cat=deprecation")` -- But it seems we should update the Scala API, too. If you cannot handle it, we can do a follow up PR.
   
   It seems, we need to add `split()` to `KStream.scala` and introduce a new `BranchedKStream.scala` and `Branch.scala` classes and maybe some translations from Java `Consumer`/`Function` to their Scala variants. But I also don't really now Scala; @vvcephei should nkow better.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548621313



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
##########
@@ -492,6 +496,16 @@ public void foreach(final ForeachAction<? super K, ? super V> action,
         return branchChildren;
     }
 
+    @Override
+    public BranchedKStream<K, V> split() {
+        return new BranchedKStreamImpl<>(this, repartitionRequired, NamedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> split(final Named named) {
+        return new BranchedKStreamImpl<>(this, repartitionRequired, new NamedInternal(named));

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547599107



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map
+ * </ul>
+ * <p>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch((key, value) -> value.contains("A"), Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch((key, value) -> value.contains("B"), Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@link BranchedKStream}

Review comment:
       `{@code BranchedKStream}`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
##########
@@ -773,10 +775,32 @@
      * @param named  a {@link Named} config used to name the processor in the topology
      * @param predicates the ordered list of {@link Predicate} instances
      * @return multiple distinct substreams of this {@code KStream}
+     * @deprecated since 2.7. Use {@link #split(Named)} instead.
      */
+    @Deprecated
     @SuppressWarnings("unchecked")
     KStream<K, V>[] branch(final Named named, final Predicate<? super K, ? super V>... predicates);
 
+    /**
+     * Splits this stream. {@link BranchedKStream} can be used for routing the records to different branches depending

Review comment:
       Nit `Split` (no `s`) -- we use imperative to write JavaDocs.
   
   `this stream` -> `this {@code KStream}.`
   
   (Same for the overload method)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r565519066



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);

Review comment:
       I agree, it seems like a good idea to check for `null` here.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map (but if the function returns `null`, that is 'no result', it's also 'swallowed'). Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further debates :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548621190



##########
File path: streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
##########
@@ -615,7 +615,7 @@ public void shouldUseSpecifiedNameForTransformValuesWithKey() {
     }
 
     @Test
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "deprecation"})

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564983682



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity
+   *              function will be used. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)
+
+  /**
+   * Create an instance of `Branched` with provided chain consumer and branch name suffix.
+   *
+   * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+   *              the respective branch will not be added to the resulting Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated

Review comment:
       as above; remove `If null, a default branch name suffix will be generated`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548642138



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);
+        return result;
+    }
+
+    private void createBranch(final Branched<K, V> branched, final int index) {
+        final BranchedInternal<K, V> branchedInternal = new BranchedInternal<>(branched);
+        final String branchChildName = branchedInternal.branchProcessorName(splitterName, index);
+        childNames.add(branchChildName);
+        source.builder.newProcessorName(branchChildName);
+        final ProcessorParameters<K, V> parameters = new ProcessorParameters<>(new PassThrough<>(), branchChildName);
+        final ProcessorGraphNode<K, V> branchChildNode = new ProcessorGraphNode<>(branchChildName, parameters);
+        source.builder.addGraphNode(splitterNode, branchChildNode);
+        final KStreamImpl<K, V> newStream = new KStreamImpl<>(branchChildName, source.keySerde,
+                source.valueSerde, source.subTopologySourceNodes,
+                repartitionRequired, branchChildNode, source.builder);
+        branchedInternal.process(newStream, branchChildName, result);

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-773549603


   Thanks, all, that Scala fix looks perfect to me.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547605439



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());

Review comment:
       Missing `null` check for `predicate` (similar below for other methods and parameters).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564988800



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result

Review comment:
       \cc @vvcephei 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548621190



##########
File path: streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
##########
@@ -615,7 +615,7 @@ public void shouldUseSpecifiedNameForTransformValuesWithKey() {
     }
 
     @Test
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "deprecation"})

Review comment:
       done, added a new corresponding test!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548645203



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to

Review comment:
       I clearly remember that something made me to write it this way, but I have to recall...




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-772153631


   Wait failure do you see? Seem Jenkins in still running. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: KIP-418 implementation

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-667352270


   Hey @mjsax , do you have time to give this a first pass?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-759130287


   @vvcephei -- hope you are also ok with the proposed changes to the KIP as per the PR description on top: https://github.com/apache/kafka/pull/9107#issuecomment-666749809


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further debates :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev edited a comment on pull request #9107: KAFKA-5488: KIP-418 implementation

Posted by GitBox <gi...@apache.org>.
inponomarev edited a comment on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-666749809


   ⚠️ Two differences with KIP specification, discussion needed⚠️ 
   
   1. Instead of multiple overloaded variants of `Branched.with` we now have `Branched.withFunction` and `Branched.withConsumer`. This is because of compiler warnings about overloading (`Function` and `Consumer` being indistinguishable when supplied as lambdas)
   
   2. 'Fully covariant' signatures like `Consumer<? super KStream<? super K, ? super V>>` don't work as expected. Used `Consumer<? super KStream<K, V>>` instead
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548521379



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Do we use dots at the end of list items? (Sorry -- English is my second language so I really don't know. I noticed that sometimes you do but in most cases you don't. In Russian language we must always use dots or commas at the end of list items, but Russian and English punctuation are completely different.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548521379



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Do we use dots at the end of list items? (Sorry -- English is my second language so I really don't know. I noticed that sometimes you do but in most times you don't. In Russian language we always must use dots or commas at the end of list items, but Russian and English punctuation are completely different.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548521379



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Do we use dots at the end of list items? (Sorry -- English is my second language so I really don't know. I noticed that sometimes you do but in most times you don't. In Russian language we always do, but Russian and English punctuation are completely different.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515526



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,

Review comment:
       see https://github.com/apache/kafka/pull/9107#discussion_r548515181




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-773692994


   Merged to `trunk`.
   
   Congrats for getting this into the 2.8.0 release @inponomarev -- great work!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-759644375


   @vvcephei @mjsax I added full Scala wrapper for the new API: `split` method, `BranchedKStream` and `Branched`. Also added Scala unit tests that verify main use cases


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r568413963



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)

Review comment:
       Instead of using `asJava` we can do those three lines as:
   ```
       BranchedJ.withFunction((f: KStreamJ[K, V]) => chain.apply(new KStream[K, V](f)).inner, name)
   ```

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._

Review comment:
       Based on you main comment, you are right, that those `FunctionConverters ` are only available, in Scala 2.13, but the code must compile with Scala 2.12.
   
   Thus we must remove this import and do the `Function/Consumer` conversion below manually (cf below).

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)
+
+  /**
+   * Create an instance of `Branched` with provided chain consumer and branch name suffix.
+   *
+   * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+   *              the respective branch will not be added to the resulting Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withConsumer(java.util.function.Consumer, java.lang.String)`
+   */
+  def withConsumer[K, V](chain: KStream[K, V] => Unit, name: String = null): BranchedJ[K, V] =
+    BranchedJ.withConsumer({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s))
+    }.asJava, name)

Review comment:
       Instead of using `asJava` we can do those three lines as:
   ```
       BranchedJ.withConsumer((c: KStreamJ[K, V]) => chain.apply(new KStream[K, V](c)), name)
   ```

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._

Review comment:
       Based on your comment, you are right, that those `FunctionConverters ` are only available in Scala 2.13, but the code must compile with Scala 2.12.
   
   Thus we must remove this import and do the `Function/Consumer` conversion below manually (cf below).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-752730961


   Maybe I miss-understood you question. I thought the build fails because we are using some deprecated method -- for this case, we can make the build pass by suppressing the warning. If you want to deprecate a method in the Scala API, you just add `@deprecated` similar to Java. -- I guess it makes sense to also deprecate the `KStream.scala#branch()` method, but suppressing the warning should also make the build pass and we can deprecate this method when we add the new `split()` method.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-752098770


   > Can you also update the docs for Kafka Streams and the 2.8 upgrade guide in this PR.
   
   The documentation had been already updated (see changes in `docs/streams/developer-guide/dsl-api.html`)
   
   I also modified  `docs/upgrade.html` -- should I add something more here, like code examples?
   
   Another question: CI checks fail because of usage of deprecated `branch` method in `streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/KStream.scala`. Since I'm not a Scala user, I have no idea of what should be done here. 
   
   Most likely we should deprecate the `branch` method and add a wrapper for the new `split` method, but I don't know how to do this correctly.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548503824



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       See my reply below, where we discuss `null` consumers: https://github.com/apache/kafka/pull/9107#discussion_r548515181
   
   (in short: I agree, I think we shouldn't)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548503824



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       The KIP itself only says 'If no chain function or consumer is provided, then the value is the branch itself' which is implemented in `BranchedInternal` as following
   
   ```java
   if (chainFunction == null) {
               transformedStream = newStream;
           } else {
               transformedStream = chainFunction.apply(newStream);
           }
   ```
   
   So actually if `null` is passed in here, NPE won't happen, and JavaDoc just describes what actually happens. 
   
   But you might be right! I can't think of sanely reasons to explicitly pass in `null`s, since one can always pass  `ks->ks`. But as a result of mistake, one can unintentionally pass `null`
   
   Shall I throw NPEs explicitly here and in similar places, WDYT?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-773549603


   Thanks, all, that Scala fix looks perfect to me.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-751922810


   About the original comment: https://github.com/apache/kafka/pull/9107#issuecomment-666749809
   
   I am fine with those changes.
   
   About https://github.com/apache/kafka/pull/9107#issuecomment-751261181 -- that is a good point. Thanks for explaining. I guess it's a "philosophical" question if we want to allow this pattern though, or if we want to require that either `defaultBranch()` or `noDefaultBranch()` is called? -- I did consider calling `branch()` like a builder pattern, and the final `[noD|d]efaultBranch` call is basically `build()`?
   
   Curious to hear what @vvcephei thinks about it.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564973026



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.

Review comment:
       `Define` (no `s` -- we use imperative in JavaDocs)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547587938



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.

Review comment:
       nit: `{@link ' -> `{@code` (we should use `@link` to link to other classes but use `@code` to refer to ourself -- there is no point in letting a JavaDoc page link to itself.
   
   Similar below.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {

Review comment:
       nit: can you first have all the static builder methods and the instance methods at the end of the class?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;

Review comment:
       nit: should be one line (easier to read)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       nit `supposed` -> `used` ?
   
   `this function` -> `the provided function`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       `If {@code null} is passed instead of an actual consumer, the branch will be added unmodified to the resulting {@code Map}.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).

Review comment:
       Maybe add a note that not all branches are added to the `Map`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead

Review comment:
       `you can apply [multiple] {#filter} [operators], one for each predicate, instead

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed
+     *              and the branch will be added to the resulting {@code Map}.
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        return new Branched<K, V>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function and branch name postfix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       used
   
   If the provided function

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via

Review comment:
       `If a [branch?] name is provided [in] {@link BranchedKStream#branch(Predicate, Branched)} via the {@link Branched} parameter`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used

Review comment:
       nit: missing `.` at the end

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via

Review comment:
       `is provided [in]`
   
   `via [the] {@link Branched} parameter.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is

Review comment:
       `and returns` -> `that returns` (or `and it returns`)
   
   `value` -> `{@link KStream}` ? (and `than the returned {@link KStream}` is added to the {@code Map}.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)

Review comment:
       Do we need the content in the parentheses?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}

Review comment:
       `then the value is the branch itself` -> ` then the branch itself is added to the {@code Map}` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function

Review comment:
       nit: missing "." (some for other bullet points)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.

Review comment:
       `the respective entry is not put to the map.` -> `, then no entry is added to the map.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);
+        return result;
+    }
+
+    private void createBranch(final Branched<K, V> branched, final int index) {
+        final BranchedInternal<K, V> branchedInternal = new BranchedInternal<>(branched);
+        final String branchChildName = branchedInternal.branchProcessorName(splitterName, index);

Review comment:
       As we call `branchProcessorName` only once, I am wondering if we should embed the code here and remove `branchProcessorName` -- our usual pattern is to only have getters on the `XxxInternal` implemenation.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,

Review comment:
       As above, should we even allow a not-null consumer?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map
+ * </ul>

Review comment:
       Seems the `null` consumer case is missing (is case we want to allow if, but as mentioned above, I have some doubts)?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map
+ * </ul>
+ * <p>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch((key, value) -> value.contains("A"), Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch((key, value) -> value.contains("B"), Branched.withConsumer(ks -> ks.to("B")))

Review comment:
       To simplify the example, should we use `predicate1`, `predicate2` as in the first example?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
##########
@@ -773,10 +775,32 @@
      * @param named  a {@link Named} config used to name the processor in the topology
      * @param predicates the ordered list of {@link Predicate} instances
      * @return multiple distinct substreams of this {@code KStream}
+     * @deprecated since 2.7. Use {@link #split(Named)} instead.
      */
+    @Deprecated
     @SuppressWarnings("unchecked")
     KStream<K, V>[] branch(final Named named, final Predicate<? super K, ? super V>... predicates);
 
+    /**
+     * Splits this stream. {@link BranchedKStream} can be used for routing the records to different branches depending

Review comment:
       Nit `Split` (no `s`) -- we use imperative to write JavaDocs.
   
   `this stream` -> `this {@code KStream}.
   
   (Same for the overload method)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
##########
@@ -492,6 +496,16 @@ public void foreach(final ForeachAction<? super K, ? super V> action,
         return branchChildren;
     }
 
+    @Override
+    public BranchedKStream<K, V> split() {
+        return new BranchedKStreamImpl<>(this, repartitionRequired, NamedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> split(final Named named) {
+        return new BranchedKStreamImpl<>(this, repartitionRequired, new NamedInternal(named));

Review comment:
       missing `null` check for `named`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java
##########
@@ -41,14 +43,18 @@
     private class KStreamBranchProcessor extends AbstractProcessor<K, V> {
         @Override
         public void process(final K key, final V value) {
-            for (int i = 0; i < predicates.length; i++) {
-                if (predicates[i].test(key, value)) {
+            for (int i = 0; i < predicates.size(); i++) {
+                if (predicates.get(i).test(key, value)) {
                     // use forward with child here and then break the loop
                     // so that no record is going to be piped to multiple streams
-                    context().forward(key, value, To.child(childNodes[i]));
-                    break;
+                    context().forward(key, value, To.child(childNodes.get(i)));
+                    return;
                 }
             }
+            // using default child node if supplied

Review comment:
       Should we really do it this way, or add a predicate to the list that always returns `true` if a default branch is added?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed
+     *              and the branch will be added to the resulting {@code Map}.
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        return new Branched<K, V>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function and branch name postfix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain, final String name) {

Review comment:
       nit: formatting
   ```
   final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain,
   final String name) {
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,

Review comment:
       `If a non-null branch is provided here`? (`branch` -> `consumer`?)
   
   But I would propose to simplify it, and just use `By default` (as passing in a non-null consumer should be the "default" usage).

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed
+     *              and the branch will be added to the resulting {@code Map}.
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        return new Branched<K, V>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function and branch name postfix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain, final String name) {
+        return new Branched<>(name, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function and branch name postfix.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,

Review comment:
       as above

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>

Review comment:
       nit: Do we need a new paragraph? We get a list below anyway.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}

Review comment:
       nit: missing `.` at the end

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       nit: missing `.` at the end

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>

Review comment:
       nit: Do we need a new paragraph after a list?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map
+ * </ul>
+ * <p>

Review comment:
       as above

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>

Review comment:
       as above

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to

Review comment:
       I am wondering if it might be better to move this code into a `build` method that would be called within `defaultBranch()` / `noDefaultBranch()` ?
   
   The pattern to pass in empty list that we modify later seems undesirable, and we should first build the list, and than pass them in -- otherwise, we make assumptions how `ProcessorParameters` and `ProcessorGraphNode` might be implemented what we should avoid.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);

Review comment:
       Should we call `branch((k,v) -> true, branched)` instead to just add a predicate and branch? This way, the default branch is nothing special at runtime any longer.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       Actually, I am wondering if we should allow to pass in `null`? Thoughts?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();

Review comment:
       `result` -> `outputBranches` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.Map;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String branchProcessorName(final String prefix, final int index) {
+        if (name == null) {
+            return prefix + index;
+        } else {
+            return prefix + name;
+        }
+    }
+
+    public void process(final KStreamImpl<K, V> newStream, final String branchChildName, final Map<String, KStream<K, V>> result) {
+        final KStream<K, V> transformedStream;
+        if (chainFunction == null) {
+            transformedStream = newStream;
+        } else {
+            transformedStream = chainFunction.apply(newStream);
+        }
+        if (transformedStream == null) {
+            return;
+        }
+        if (chainConsumer != null) {
+            chainConsumer.accept(transformedStream);
+            return;
+        } else {
+            result.put(branchChildName, transformedStream);
+        }
+    }

Review comment:
       I think this method hard to read. Proposal:
   ```
   if (chainFunction != null) {
     final KStream<K, V> transformedStream = chainFunction.apply(newStream);
     if (transformedStream != null) {
       result.put(branchChildName, transformedStream);
     }
   } else if (chainConsumer != null) {
     chainConsumer.accept(transformedStream);
   } else {
     result.put(branchChildName, newStream);
   }
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       I tried to read up on the KIP discussion thread, and I am wondering if we did agree this this behavior? My understanding was that if a consumer is use, there won't be any entry in the `Map` for this branch?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map

Review comment:
       `If a [non-null] consumer`
   
   `then the the respective entry is not put to the map` -> `, then no entry is added to the map.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);
+        return result;
+    }
+
+    private void createBranch(final Branched<K, V> branched, final int index) {
+        final BranchedInternal<K, V> branchedInternal = new BranchedInternal<>(branched);
+        final String branchChildName = branchedInternal.branchProcessorName(splitterName, index);
+        childNames.add(branchChildName);
+        source.builder.newProcessorName(branchChildName);
+        final ProcessorParameters<K, V> parameters = new ProcessorParameters<>(new PassThrough<>(), branchChildName);
+        final ProcessorGraphNode<K, V> branchChildNode = new ProcessorGraphNode<>(branchChildName, parameters);
+        source.builder.addGraphNode(splitterNode, branchChildNode);
+        final KStreamImpl<K, V> newStream = new KStreamImpl<>(branchChildName, source.keySerde,
+                source.valueSerde, source.subTopologySourceNodes,
+                repartitionRequired, branchChildNode, source.builder);
+        branchedInternal.process(newStream, branchChildName, result);

Review comment:
       Same comment as for `branchProcessorName` -- it seems better to keep `BranchedInternal` a simple container and pull in the code into this method.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.Map;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String branchProcessorName(final String prefix, final int index) {
+        if (name == null) {
+            return prefix + index;
+        } else {
+            return prefix + name;
+        }
+    }
+
+    public void process(final KStreamImpl<K, V> newStream, final String branchChildName, final Map<String, KStream<K, V>> result) {

Review comment:
       `newStream` -> `branch` ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)
+ *     <li>If a chain function is provided and returns a non-null value for a given branch, then the value is
+ *     the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then the respective entry is not put to the map.
+ *     <li>If a consumer is provided for a given branch, then the the respective entry is not put to the map
+ * </ul>
+ * <p>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch((key, value) -> value.contains("A"), Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch((key, value) -> value.contains("B"), Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@link BranchedKStream}

Review comment:
       `{@code BranchedKStream}

##########
File path: streams/src/test/java/org/apache/kafka/streams/StreamsBuilderTest.java
##########
@@ -615,7 +615,7 @@ public void shouldUseSpecifiedNameForTransformValuesWithKey() {
     }
 
     @Test
-    @SuppressWarnings("unchecked")
+    @SuppressWarnings({"unchecked", "deprecation"})

Review comment:
       Seems we should add a corresponding test for `split()` ? (Same for other test classes.)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.

Review comment:
       Add: `If a record does not match any predicates, it will be routed to the default branch, or dropped if no default branch is created.`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());

Review comment:
       Missing `null` check for `predicate` (similar below for other methods and parameters.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r568413963



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)

Review comment:
       Instead of using `asJava` we can do those three lines as:
   ```
       BranchedJ.withFunction((f: KStreamJ[K, V]) => chain.apply(new KStream[K, V](f)).inner, name)
   ```

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._

Review comment:
       Based on you main comment, you are right, that those `FunctionConverters ` are only available, in Scala 2.13, but the code must compile with Scala 2.12.
   
   Thus we must remove this import and do the `Function/Consumer` conversion below manually (cf below).

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)
+
+  /**
+   * Create an instance of `Branched` with provided chain consumer and branch name suffix.
+   *
+   * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+   *              the respective branch will not be added to the resulting Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withConsumer(java.util.function.Consumer, java.lang.String)`
+   */
+  def withConsumer[K, V](chain: KStream[K, V] => Unit, name: String = null): BranchedJ[K, V] =
+    BranchedJ.withConsumer({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s))
+    }.asJava, name)

Review comment:
       Instead of using `asJava` we can do those three lines as:
   ```
       BranchedJ.withConsumer((c: KStreamJ[K, V]) => chain.apply(new KStream[K, V](c)), name)
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r568681712



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)

Review comment:
       Hmmm, this looks prettier :-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549555914



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String getName() {

Review comment:
       It's a naming convention in the whole Kafka code base, to omit the `get` prefix for all getter methods, ie, this should be `name()`. (Similar below for the other getters.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564987769



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);

Review comment:
       \cc @vvcephei 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564768542



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");
+        return new Branched<>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function and branch name suffix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be used. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain, final String name) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(name, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer and branch name suffix.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<? super KStream<K, V>> chain,
+                                                     final String name) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");

Review comment:
       nit: `should not` -> `cannot` (or `can't` if you prefer)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-761803551


   Hi @vvcephei thank you for your commits! Is everything else OK, especially https://github.com/apache/kafka/pull/9107#issuecomment-666749809?
   
   @mjsax I pushed small fixes to Javadoc/Scaladoc, and AFAICS only tests not related to the changes are failing.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548629205



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);

Review comment:
       The default branch should have index 0 (so it will be stable when branches are added or removed), but it should always be checked after all other branches. And when we come to the default branch during message processing, there is actually no need in dereferncing a predicate and calling `test`... that's why I treat the default branch differently.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. When we provide **nothing**, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split().branch(isEven).branch(isMultipleOfSeven);
   ```
   we agreed to treat this like 
   
   ```java
           BranchedKStream<Integer, String> branch = source.split()
                   .branch(isEven, Branched.withFunction(Function.identity()))
                   .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me this dirty trick -- thus `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to just prohibit passing in nulls and throw explicit NPEs for both functions and consumers. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: KIP-418 implementation

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-666749809


   ⚠️ Two differences with KIP specification⚠️ 
   
   1. Instead of multiple overloaded variants of `Branched.with` we now have `Branched.withFunction` and `Branched.withConsumer`. This is because of compiler warnings about overloading (`Function` and `Consumer` being indistinguishable when supplied as lambdas)
   
   2. 'Fully covariant' signatures like `Consumer<? super KStream<? super K, ? super V>>` don't work as expected. Used `Consumer<? super KStream<K, V>>` instead
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-771925298


   Hi @mjsax , I have rebased and manually merged conflicts, and also removed `FunctionConverters`
   
   JDK8 build still fails, but this time much later -- something related to integration testing


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564983488



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity
+   *              function will be used. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated

Review comment:
       remove `If null, a default branch name suffix will be generated` -- `null` should not be allowed (cf my comment on the corresponding Java class)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r562212694



##########
File path: docs/upgrade.html
##########
@@ -21,6 +21,12 @@
 
 <h5><a id="upgrade_280_notable" href="#upgrade_280_notable">Notable changes in 2.8.0</a></h5>
 
+<ul>
+    <li>Kafka Streams introduce a type-safe split() operator as a substitution for deprecated KStream#branch method

Review comment:
       nit: `split()` -> `<code>split()</code>`
   Same for `KStream#branch` (should we add `()`, too?)

##########
File path: docs/streams/developer-guide/dsl-api.html
##########
@@ -366,24 +366,28 @@ <h4 class="anchor-heading"><a id="streams_concepts_globalktable" class="anchor-l
                         </ul>
                     </td>
                         <td><p class="first">Branch (or split) a <code class="docutils literal"><span class="pre">KStream</span></code> based on the supplied predicates into one or more <code class="docutils literal"><span class="pre">KStream</span></code> instances.
-                            (<a class="reference external" href="/{{version}}/javadoc/org/apache/kafka/streams/kstream/KStream.html#branch-org.apache.kafka.streams.kstream.Predicate...-">details</a>)</p>
+                            (<a class="reference external" href="/{{version}}/javadoc/org/apache/kafka/streams/kstream/KStream.html#split">details</a>)</p>
                             <p>Predicates are evaluated in order.  A record is placed to one and only one output stream on the first match:
                                 if the n-th predicate evaluates to true, the record is placed to n-th stream.  If no predicate matches, the
                                 the record is dropped.</p>
                             <p>Branching is useful, for example, to route records to different downstream topics.</p>
-                            <div class="last highlight-java"><div class="highlight"><pre><span></span><span class="n">KStream</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;</span> <span class="n">stream</span> <span class="o">=</span> <span class="o">...;</span>
-<span class="n">KStream</span><span class="o">&lt;</span><span class="n">String</span><span class="o">,</span> <span class="n">Long</span><span class="o">&gt;[]</span> <span class="n">branches</span> <span class="o">=</span> <span class="n">stream</span><span class="o">.</span><span class="na">branch</span><span class="o">(</span>
-    <span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-&gt;</span> <span class="n">key</span><span class="o">.</span><span class="na">startsWith</span><span class="o">(</span><span class="s">&quot;A&quot;</span><span class="o">),</span> <span class="cm">/* first predicate  */</span>
-    <span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-&gt;</span> <span class="n">key</span><span class="o">.</span><span class="na">startsWith</span><span class="o">(</span><span class="s">&quot;B&quot;</span><span class="o">),</span> <span class="cm">/* second predicate */</span>
-    <span class="o">(</span><span class="n">key</span><span class="o">,</span> <span class="n">value</span><span class="o">)</span> <span class="o">-&gt;</span> <span class="kc">true</span>                 <span class="cm">/* third predicate  */</span>
-  <span class="o">);</span>
-
-<span class="c1">// KStream branches[0] contains all records whose keys start with &quot;A&quot;</span>
-<span class="c1">// KStream branches[1] contains all records whose keys start with &quot;B&quot;</span>
-<span class="c1">// KStream branches[2] contains all other records</span>
-
-<span class="c1">// Java 7 example: cf. `filter` for how to create `Predicate` instances</span></code></pre></div>
-                            </div>
+                            <pre class="brush: java;">
+KStream<String, Long> stream = ...;
+Map<String, KStream<String, Long>> branches = 
+stream.split(Named.as("Branch-"))
+.branch((key, value) -> key.startsWith("A"),  /* first predicate  */ 
+Branched.as("A")) 
+.branch((key, value) -> key.startsWith("B"),  /* second predicate */
+Branched.as("B"))  
+.defaultBranch(Branched.as("C"))
+);

Review comment:
       nit: seems we use some indention for this code block?
   ```
   Map<String, KStream<String, Long>> branches =
       stream.split(Named.as("Branch-"))
             .branch((key, value) -> key.startsWith("A"),  /* first predicate  */ 
                     Branched.as("A")) 
             .branch((key, value) -> key.startsWith("B"),  /* second predicate */
                     Branched.as("B"))  
           .defaultBranch(Branched.as("C"))
   );
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");
+        return new Branched<>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function and branch name suffix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be used. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain, final String name) {
+        Objects.requireNonNull(chain, "chain function should not be null");

Review comment:
       nit: `should not` -> `cannot`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need

Review comment:
       nit: `{@code true}`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);

Review comment:
       Missing `null` check for `name`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch())
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @param branched A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                 branch consumer or branch mapper for [[BranchedKStream]].
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(branched: Branched[K, V]): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch(branched))
+
+  /**
+   * Finalizes the construction of branches without forming a default branch.

Review comment:
       Add: `Calling [[noDefaultBranch]] or [[defaultBranch]] is optional.`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch())
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted

Review comment:
       nit: `Finalize` (no `s`)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");

Review comment:
       nit: `should not` -> `cannot` 

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,

Review comment:
       nit: remove `here`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");
+        return new Branched<>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function and branch name suffix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity

Review comment:
       remove ` If {@code null}, the identity {@code kStream -> kStream} function will be used.` -- we don't allow `null`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>

Review comment:
       nit: add empty line above headline for better readability in the code (won't change the produced html).

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");

Review comment:
       nit: `should not` -> `cannot`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");
+        return new Branched<>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function and branch name suffix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be used. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain, final String name) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(name, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer and branch name suffix.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,

Review comment:
       nit: remove `here`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");
+        return new Branched<>(null, null, chain);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function and branch name suffix.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be used. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain, final String name) {
+        Objects.requireNonNull(chain, "chain function should not be null");
+        return new Branched<>(name, chain, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain consumer and branch name suffix.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param name  the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *              (see {@link BranchedKStream} description for details)
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withConsumer(final Consumer<? super KStream<K, V>> chain,
+                                                     final String name) {
+        Objects.requireNonNull(chain, "chain consumer should not be null");

Review comment:
       nit: `should not` -> `cannot`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,

Review comment:
       proposal (add) `operators[ to the same {@link KStream} instance],`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result

Review comment:
       nit: `first match` (without dash) ?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map

Review comment:
       Should we merge this sentence to the previous bullet point, instead of making a new bullet point?

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map

Review comment:
       Remove `non-null` (`null` is not allowed anyway)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value

Review comment:
       `If a non-null chain function` -> `If a chain function`
   
   (We don't allow `null` as chain function.)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.

Review comment:
       `Define` (no `s` -- we use imperative in JavaDocs.

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.

Review comment:
       as above

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}

Review comment:
       I think it's worth to highlight that `defaultBranch / noDefaultBranch` are optional (to avoid the confusion about "builder pattern"):
   
   `by an optional {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}  method call (see ...).`

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.

Review comment:
       Add:
   ```
   Calling {@code defaultBranch()} or {@link #noDefaultBranch()} is optional.
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted

Review comment:
       `Finalize` (no `s')

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted

Review comment:
       nit: `Finalize` (no `s`)

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch())
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @param branched A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                 branch consumer or branch mapper for [[BranchedKStream]].
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(branched: Branched[K, V]): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch(branched))
+
+  /**
+   * Finalizes the construction of branches without forming a default branch.

Review comment:
       nit: `Finalize` (no `s`)

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity

Review comment:
       Remove `If `null`, the identity function will be used.` (we don't allow `null`)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@code BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch();
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted

Review comment:
       as above

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by

Review comment:
       nit:
   ```
   Map -> `Map`
   ```
   
   add:
   ```
   by [an optional] defaultBranch or noDefaultBranch [method call].
   ```

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity
+   *              function will be used. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated

Review comment:
       remove `If `null`, a default branch name suffix will be generated` -- `null` should not be allowed (cf my comment on the corresponding Java class)

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity
+   *              function will be used. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)
+
+  /**
+   * Create an instance of `Branched` with provided chain consumer and branch name suffix.
+   *
+   * @param chain A consumer to which the branch will be sent. If a non-null consumer is provided here,
+   *              the respective branch will not be added to the resulting Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated

Review comment:
       as above; remove `If `null`, a default branch name suffix will be generated`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,

Review comment:
       nit:
   `operators[ to the same [[KStream]] instance],`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.

Review comment:
       Add: `Calling [[defaultBranch]] or [[noDefaultBranch]] is optional.`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result

Review comment:
       `first match` (no dash?)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@code BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch();
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.

Review comment:
       Add:
   ```
   Calling {@code defaultBranch()} or {@link #noDefaultBranch()} is optional.
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@code BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch();
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @param branched A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                 branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                 for {@link BranchedKStream})
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@link BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch(Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches without forming a default branch.

Review comment:
       Add:
   ```
   Calling {@code noDefaultBranch()} or {@link #defaultBranch()} is optional.
   ```

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
##########
@@ -773,10 +775,32 @@
      * @param named  a {@link Named} config used to name the processor in the topology
      * @param predicates the ordered list of {@link Predicate} instances
      * @return multiple distinct substreams of this {@code KStream}
+     * @deprecated since 2.7. Use {@link #split(Named)} instead.

Review comment:
       `2.7` -> `2.8`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need

Review comment:
       nit:
   ```
   true -> `true`
   ```

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @param branched  A [[Branched]] parameter, that allows to define a branch name, an in-place
+   *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+   *                  for [[BranchedKStream]])
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean, branched: Branched[K, V]): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate, branched)
+    this
+  }
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.
+   *
+   * @return Map of named branches. For rules of forming the resulting map, see [[BranchedKStream]]
+   *         description.
+   */
+  def defaultBranch(): Map[String, KStream[K, V]] = toScalaMap(inner.defaultBranch())
+
+  /**
+   * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+   * by other branches.

Review comment:
       Add: `Calling [[defaultBranch]] or [[noDefaultBranch]] is optional.`

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.

Review comment:
       nit: `Define` (no `s`)

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple {@link KStream#filter(Predicate)} operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a branch name is provided in {@link BranchedKStream#branch(Predicate, Branched)} via the
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <ul>
+ *     <li>If no chain function or consumer is provided in {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     the {@link Branched} parameter, then the the branch itself is added to the {@code Map}
+ *     <li>If a non-null chain function is provided and it returns a non-null value for a given branch, then the value
+ *     is the result returned by this function
+ *     <li>If a chain function returns {@code null} for a given branch, then no entry is added to the map
+ *     <li>If a non-null consumer is provided for a given branch, then no entry is added to the map
+ * </ul>
+ * For example:
+ * <pre> {@code
+ * Map<String, KStream<..., ...>> result =
+ *   source.split(Named.as("foo-"))
+ *     .branch(predicate1, Branched.as("bar"))                    // "foo-bar"
+ *     .branch(predicate2, Branched.withConsumer(ks->ks.to("A"))  // no entry: a Consumer is provided
+ *     .branch(predicate3, Branched.withFunction(ks->null))       // no entry: chain function returns null
+ *     .branch(predicate4)                                        // "foo-4": name defaults to the branch position
+ *     .defaultBranch()                                           // "foo-0": "0" is the default name for the default branch
+ * }</pre>
+ *
+ * <h2><a name="examples">Usage examples</a></h2>
+ *
+ * <h3>Direct Branch Consuming</h3>
+ * In many cases we do not need to have a single scope for all the branches, each branch being processed completely
+ * independently from others. Then we can use 'consuming' lambdas or method references in {@link Branched} parameter:
+ *
+ * <pre> {@code
+ * source.split()
+ *     .branch(predicate1, Branched.withConsumer(ks -> ks.to("A")))
+ *     .branch(predicate2, Branched.withConsumer(ks -> ks.to("B")))
+ *     .defaultBranch(Branched.withConsumer(ks->ks.to("C")));
+ * }</pre>
+ *
+ * <h3>Collecting branches in a single scope</h3>
+ * In other cases we want to combine branches again after splitting. The map returned by
+ * {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} methods provides
+ * access to all the branches in the same scope:
+ *
+ * <pre> {@code
+ * Map<String, KStream<String, String>> branches = source.split(Named.as("split-"))
+ *     .branch((key, value) -> value == null, Branched.withFunction(s -> s.mapValues(v->"NULL"), "null")
+ *     .defaultBranch(Branched.as("non-null"));
+ *
+ * KStream<String, String> merged = branches.get("split-non-null").merge(branches.get("split-null"));
+ * }</pre>
+ *
+ * <h3>Dynamic branching</h3>
+ * There is also a case when we might need to create branches dynamically, e. g. one per enum value:
+ *
+ * <pre> {@code
+ * BranchedKStream branched = stream.split();
+ * for (RecordType recordType : RecordType.values())
+ *     branched.branch((k, v) -> v.getRecType() == recordType,
+ *         Branched.withConsumer(recordType::processRecords));
+ * }</pre>
+ *
+ * @param <K> Type of keys
+ * @param <V> Type of values
+ * @see KStream
+ */
+public interface BranchedKStream<K, V> {
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate);
+
+    /**
+     * Defines a branch for records that match the predicate.
+     *
+     * @param predicate A {@link Predicate} instance, against which each record will be evaluated.
+     *                  If this predicate returns {@code true} for a given record, the record will be
+     *                  routed to the current branch and will not be evaluated against the predicates
+     *                  for the remaining branches.
+     * @param branched  A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                  branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                  for {@link BranchedKStream})
+     * @return {@code this} to facilitate method chaining
+     */
+    BranchedKStream<K, V> branch(Predicate<? super K, ? super V> predicate, Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@code BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch();
+
+    /**
+     * Finalizes the construction of branches and defines the default branch for the messages not intercepted
+     * by other branches.
+     *
+     * @param branched A {@link Branched} parameter, that allows to define a branch name, an in-place
+     *                 branch consumer or branch mapper (see <a href="#examples">code examples</a>
+     *                 for {@link BranchedKStream})
+     * @return {@link Map} of named branches. For rules of forming the resulting map, see {@link BranchedKStream}
+     * <a href="#maprules">description</a>.
+     */
+    Map<String, KStream<K, V>> defaultBranch(Branched<K, V> branched);
+
+    /**
+     * Finalizes the construction of branches without forming a default branch.

Review comment:
       as above

##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated

Review comment:
       remove: `If {@code null}, a default branch name suffix will be generated` (we should not allow `null` to align to other methods, eg, `Named.as()`)

##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/BranchedKStream.scala
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import java.util
+
+import org.apache.kafka.streams.kstream
+import org.apache.kafka.streams.kstream.{BranchedKStream => BranchedKStreamJ}
+import org.apache.kafka.streams.scala.FunctionsCompatConversions.PredicateFromFunction
+
+import scala.jdk.CollectionConverters._
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with [[branch]] or [[defaultBranch]] methods. Each record is evaluated against the predicates
+ * supplied via [[Branched]] parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to `true`. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ *
+ * Each branch (which is a [[KStream]] instance) then can be processed either by
+ * a function or a consumer provided via a [[Branched]]
+ * parameter. If certain conditions are met, it also can be accessed from the Map returned by
+ * [[defaultBranch]] or [[noDefaultBranch]].
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can apply multiple [[KStream.filter]] operators,
+ * one for each predicate, instead of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ *
+ * @tparam K Type of keys
+ * @tparam V Type of values
+ */
+class BranchedKStream[K, V](val inner: BranchedKStreamJ[K, V]) {
+
+  /**
+   * Defines a branch for records that match the predicate.
+   *
+   * @param predicate A predicate against which each record will be evaluated.
+   *                  If this predicate returns `true` for a given record, the record will be
+   *                  routed to the current branch and will not be evaluated against the predicates
+   *                  for the remaining branches.
+   * @return `this` to facilitate method chaining
+   */
+  def branch(predicate: (K, V) => Boolean): BranchedKStream[K, V] = {
+    inner.branch(predicate.asPredicate)
+    this
+  }
+
+  /**
+   * Defines a branch for records that match the predicate.

Review comment:
       nit: `Define` (no `s`)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-751261181


   OK @mjsax concerning https://github.com/apache/kafka/pull/9107#discussion_r547607739 I remembered why it was implemented this way!
   
   The problem is that **it is not necessary to invoke `defaultBranch()` / `noDefaultBranch()`** when we use consumers, like in this simple example (I just added a new unit test for this case):
    
   ```java
   source.split()
       .branch(isCoffee, Branched.withConsumer(issuer::setCoffeePurchases))
       .branch(isElectronics, Branched.withConsumer(issuer::setElectronicsPurchases));
   ``` 
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548521379



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Do we use dots at the end of list items? (Sorry -- English is my second language so I really don't know. I noticed that sometimes you do but in most cases you don't. In Russian language we must always use dots or commas at the end of list items, but Russian and English punctuations are completely different.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549538598



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Also my second language... I think, it is mostly a matter of style. Personally, I prefer to use `.` at the end, because a bullet point is still a sentence from my POV, and actually, a bullet point could be multiple sentences. You also capitalized the first word of each bullet point, so it seems to be a sentence. -- I guess some people don't capitalize the first word and also don't use a `.` at the en, ie, us a "no-sentence" bullet point style. But this only work for "single sentence" bullet points.
   
   It's really a nit. I just raised my personal (an obviously very subjective) preference.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547597683



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       I tried to read up on the KIP discussion thread, and I am wondering if we did agree to this behavior? My understanding was that if a consumer is use, there won't be any entry in the `Map` for this branch?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548503824



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       The KIP itself only says 'If no chain function or consumer is provided, then the value is the branch itself' which is implemented in `BranchedInternal` as following
   
   ```java
   if (chainFunction == null) {
               transformedStream = newStream;
           } else {
               transformedStream = chainFunction.apply(newStream);
           }
   ```
   
   So actually if `null` is passed in here, NPE won't happen, and JavaDoc just describes what actually happens. 
   
   But you might be right and I can throw NPEs explicitly here when the argument is null to force people to use `ks->ks` instead of `null`. See also my comments below




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549543943



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       SGTM. In general, even if possible, I don't like to give `null` semantics :) 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-764879234


   Thanks, @inponomarev !
   
   Yes, we sometimes have to update the KIP as the PR gets finalized. I like to send a quick reply to the vote thread on the mailing list to summarize the design changes, in case anyone wants to object (I don't think I've ever had someone object).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-762937808


   Ooops, I just noticed one more thing to fix: https://github.com/apache/kafka/blob/3b7dda459ca9a1bcea72ca981c2a19a4ccffe9f2/streams/src/test/java/org/apache/kafka/streams/kstream/internals/graph/StreamsGraphTest.java#L115
   
   Will push a new commit today or tomorrow!
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-756855647


   Hey @inponomarev and @mjsax ! I'm glad to see this is moving along.
   
   Regarding https://github.com/apache/kafka/pull/9107#issuecomment-751922810 :
   
   My understanding was that defaultBranch/noDefaultBranch were the terminal operators, in that they close out the context of a BranchedKStream, and you can't add any more branches after one of those methods.
   
   But also, the whole branching construct is an incremental builder like the rest of the Kafka Streams API. In other words, just like this is a valid program:
   
   ```java
   builder.stream("input")
          .filter(myPredicate)
   ```
   
   so would be Ivan's example:
   
   ```java
   builder.split()
          .branch("myBranch", ...)
   ```
   
   What I mean by "incremental builder" is that each time you call a chained method in the DSL, it immediately adds nodes to the program, as opposed to having to call any kind of `build()` method to actually add stuff to the program. I think there are pros and cons to this design, but it seems more in line with the rest of the DSL not to require the terminal operators.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-771925298


   Hi @mjsax , I have rebased and manually merged conflicts, and also removed `FunctionConverters`
   
   JDK8 build still fails, but this time much later -- something related to integration testing


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549542995



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to

Review comment:
       Would love to learn about it. -- In general, it's easier to follow the same pattern throughout the code base. It easier to reason about the code that way, and also easier for people to learn the code base.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-772719099


   Ah I see -- well, we do have from flaky tests, so nothing to worry about I guess. The last run timed out, so I retriggered the build. However, I could build it locally with Java8/Scale 2.12 and so I guess we can merge. Just waiting for @vvcephei to take a quick look at the last Scala commit.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548606250



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.Map;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String branchProcessorName(final String prefix, final int index) {
+        if (name == null) {
+            return prefix + index;
+        } else {
+            return prefix + name;
+        }
+    }
+
+    public void process(final KStreamImpl<K, V> newStream, final String branchChildName, final Map<String, KStream<K, V>> result) {
+        final KStream<K, V> transformedStream;
+        if (chainFunction == null) {
+            transformedStream = newStream;
+        } else {
+            transformedStream = chainFunction.apply(newStream);
+        }
+        if (transformedStream == null) {
+            return;
+        }
+        if (chainConsumer != null) {
+            chainConsumer.accept(transformedStream);
+            return;
+        } else {
+            result.put(branchChildName, transformedStream);
+        }
+    }

Review comment:
       Agreed, replaced the code as you suggested




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split().branch(isEven).branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
           BranchedKStream<Integer, String> branch = source.split()
                   .branch(isEven, Branched.withFunction(Function.identity()))
                   .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me this dirty trick -- thus `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to just prohibit passing in nulls and throw explicit NPEs for both functions and consumers. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r568681712



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If the provided function returns
+   *              `null`, its result is ignored, otherwise it is added to the Map returned
+   *              by [[BranchedKStream.defaultBranch]] or [[BranchedKStream.noDefaultBranch]] (see
+   *              [[BranchedKStream]] description for details).
+   * @param name  the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *              (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   * @see `org.apache.kafka.streams.kstream.Branched#withFunction(java.util.function.Function, java.lang.String)`
+   */
+  def withFunction[K, V](chain: KStream[K, V] => KStream[K, V], name: String = null): BranchedJ[K, V] =
+    BranchedJ.withFunction({ s: KStreamJ[K, V] =>
+      chain.apply(new KStream[K, V](s)).inner
+    }.asJava, name)

Review comment:
       Hmmm, this looks prettier :-)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549722543



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String getName() {

Review comment:
       done




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-751926120


   @inponomarev -- Can you also update the docs for Kafka Streams and the 2.8 upgrade guide in this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-762931868


   Thank you @vvcephei, I have updated [the KIP](https://cwiki.apache.org/confluence/display/KAFKA/KIP-418%3A+A+method-chaining+way+to+branch+KStream) and now it reflects the actual implementation.
   
   I just wasn't sure if it's ok to edit specification text after it has been formally approved :-)


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev edited a comment on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev edited a comment on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-772381544


   I was talking about build 17 (triggered by Commit db573f5, see https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-9107/)
   
   Where did build 18 come from, why did it take 8 hours and then timed out -- I can't understand 😃 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-772381544


   I was talking about build 17 (triggered by Commit db573f5, see https://ci-builds.apache.org/job/Kafka/job/kafka-pr/job/PR-9107/)
   
   Where did build 18 came from, why it took 8 hours and then timed out -- I can't understand 😃 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548503824



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       The KIP itself only says 'If no chain function or consumer is provided, then the value is the branch itself' which is implemented in `BranchedInternal` as following
   
   ```java
   if (chainFunction == null) {
               transformedStream = newStream;
           } else {
               transformedStream = chainFunction.apply(newStream);
           }
   ```
   
   So actually if `null` is passed in here, NPE won't happen, and JavaDoc just describes what actually happens. 
   
   But you might be right! I can't think of sanely reasons to explicitly pass in `null`s , since we have an overloaded version of the static builder without `chain` param. But as a result of mistake, one can unintentionally pass `null`
   
   Shall I throw NPEs explicitly here and in similar places, WDYT?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Next we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to just prohibit passing in nulls and throw explicit NPEs for both functions and consumers. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax edited a comment on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax edited a comment on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-772153631


   Wait failure do you see exactly? Seem Jenkins in still running.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564766389



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,149 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Objects;
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided branch name suffix.
+     *
+     * @param name the branch name suffix to be used. If {@code null}, a default branch name suffix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@code Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If the provided function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@code Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>, ? extends KStream<K, V>> chain) {
+        Objects.requireNonNull(chain, "chain function should not be null");

Review comment:
       nit: `should not` -> `cannot`  (or `can't` if you prefer)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r568413267



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._

Review comment:
       Based on your comment, you are right, that those `FunctionConverters ` are only available in Scala 2.13, but the code must compile with Scala 2.12.
   
   Thus we must remove this import and do the `Function/Consumer` conversion below manually (cf below).




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-758218461


   Hi @vvcephei , thank you for your comment. There's another question that we were unable to solve without you -- see https://github.com/apache/kafka/pull/9107#issuecomment-752098770 from the words 'CI checks fail' and further discussion. Can you clarify, what's expected from `KStream.scala` ?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java typing system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. When we provide **nothing**, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split().branch(isEven).branch(isMultipleOfSeven);
   ```
   we agreed to treat this like 
   
   ```java
           BranchedKStream<Integer, String> branch = source.split()
                   .branch(isEven, Branched.withFunction(Function.identity()))
                   .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me this dirty trick -- thus `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to just prohibit passing in nulls and throw explicit NPEs for both functions and consumers. This, at least, can spare us from further argumets :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r547602314



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedInternal.java
##########
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.KStream;
+
+import java.util.Map;
+
+class BranchedInternal<K, V> extends Branched<K, V> {
+    BranchedInternal(final Branched<K, V> branched) {
+        super(branched);
+    }
+
+    BranchedInternal() {
+        super(null, null, null);
+    }
+
+    static <K, V> BranchedInternal<K, V> empty() {
+        return new BranchedInternal<>();
+    }
+
+    String branchProcessorName(final String prefix, final int index) {
+        if (name == null) {
+            return prefix + index;
+        } else {
+            return prefix + name;
+        }
+    }
+
+    public void process(final KStreamImpl<K, V> newStream, final String branchChildName, final Map<String, KStream<K, V>> result) {
+        final KStream<K, V> transformedStream;
+        if (chainFunction == null) {
+            transformedStream = newStream;
+        } else {
+            transformedStream = chainFunction.apply(newStream);
+        }
+        if (transformedStream == null) {
+            return;
+        }
+        if (chainConsumer != null) {
+            chainConsumer.accept(transformedStream);
+            return;
+        } else {
+            result.put(branchChildName, transformedStream);
+        }
+    }

Review comment:
       I think this method is hard to read. Proposal:
   ```
   if (chainFunction != null) {
     final KStream<K, V> transformedStream = chainFunction.apply(newStream);
     if (transformedStream != null) {
       result.put(branchChildName, transformedStream);
     }
   } else if (chainConsumer != null) {
     chainConsumer.accept(transformedStream);
   } else {
     result.put(branchChildName, newStream);
   }
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548523976



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key
+ *     <li>If a name is not provided for the branch, then the key defaults to {@code prefix + position} of the branch
+ *     as a decimal number, starting from {@code "1"}
+ *     <li>If a name is not provided for the {@link BranchedKStream#defaultBranch()} call, then the key defaults
+ *     to {@code prefix + "0"}
+ * </ul>
+ * <p>
+ * The values of the respective {@code Map<Stream, KStream<K, V>>} entries are formed as following:
+ * <p>
+ * <ul>
+ *     <li>If no chain function or consumer is provided {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, then the value is the branch itself (which is equivalent to {@code ks -> ks}
+ *     identity chain function)

Review comment:
       No we don't, removed )




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax merged pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax merged pull request #9107:
URL: https://github.com/apache/kafka/pull/9107


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549542855



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to
+        final ProcessorParameters<K, V> processorParameters =
+                new ProcessorParameters<>(new KStreamBranch<>(predicates, childNames), splitterName);
+        splitterNode = new ProcessorGraphNode<>(splitterName, processorParameters);
+        source.builder.addGraphNode(source.streamsGraphNode, splitterNode);
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate) {
+        return branch(predicate, BranchedInternal.empty());
+    }
+
+    @Override
+    public BranchedKStream<K, V> branch(final Predicate<? super K, ? super V> predicate, final Branched<K, V> branched) {
+        predicates.add(predicate);
+        createBranch(branched, predicates.size());
+        return this;
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch() {
+        return defaultBranch(BranchedInternal.empty());
+    }
+
+    @Override
+    public Map<String, KStream<K, V>> defaultBranch(final Branched<K, V> branched) {
+        createBranch(branched, 0);

Review comment:
       I guess it's fine both ways. -- The point about the index is a good one that I missed. But would still be doable I guess.
   
   I don't think that there would be any measurable runtime difference if you use a "default predicate" (what we also do in the current implementation) -- the code is just a little "cleaner" as we don't need an extra "if" at the end -- but it's also not the end of the world as the `process` method is fairly simply anyway.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Then we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   is to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further debates :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe branch() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r549543247



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/internals/BranchedKStreamImpl.java
##########
@@ -0,0 +1,97 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream.internals;
+
+import org.apache.kafka.streams.kstream.Branched;
+import org.apache.kafka.streams.kstream.BranchedKStream;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Predicate;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorGraphNode;
+import org.apache.kafka.streams.kstream.internals.graph.ProcessorParameters;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+public class BranchedKStreamImpl<K, V> implements BranchedKStream<K, V> {
+
+    private static final String BRANCH_NAME = "KSTREAM-BRANCH-";
+
+    private final KStreamImpl<K, V> source;
+    private final boolean repartitionRequired;
+    private final String splitterName;
+    private final Map<String, KStream<K, V>> result = new HashMap<>();
+
+    private final List<Predicate<? super K, ? super V>> predicates = new ArrayList<>();
+    private final List<String> childNames = new ArrayList<>();
+    private final ProcessorGraphNode<K, V> splitterNode;
+
+    BranchedKStreamImpl(final KStreamImpl<K, V> source, final boolean repartitionRequired, final NamedInternal named) {
+        this.source = source;
+        this.repartitionRequired = repartitionRequired;
+        this.splitterName = named.orElseGenerateWithPrefix(source.builder, BRANCH_NAME);
+
+        // predicates and childNames are passed by reference so when the user adds a branch they get added to

Review comment:
       Just saw your other comment: https://github.com/apache/kafka/pull/9107#issuecomment-751261181




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548515181



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns
+     *              {@code null}, its result is ignored, otherwise it is added to the {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details).
+     * @param <K>   key type
+     * @param <V>   value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> withFunction(
+            final Function<? super KStream<K, V>,
+                    ? extends KStream<K, V>> chain) {
+        return new Branched<>(null, chain, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain consumer.
+     *
+     * @param chain A consumer to which the branch will be sent. If a non-null branch is provided here,
+     *              the respective branch will not be added to the resulting {@code Map} returned
+     *              by {@link BranchedKStream#defaultBranch()} or {@link BranchedKStream#noDefaultBranch()} (see
+     *              {@link BranchedKStream} description for details). If {@code null}, a no-op consumer will be supposed

Review comment:
       OK let me explain how I understood this: 
   
   Let's try to forget about Java type system and look at the problem conceptually, from set / category theory point of view
   
   We're using either **function** or **consumer** or **nothing** for branch processing. From mathematical point of view, we can consider consumer to be a function that maps its domain to the empty set. This explains our decision why consumer 'swallows' branch unlike function that 'forwards' its result to the resulting map. Then we agreed that the case when **nothing** is provided, like in this example
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
             .branch(isEven)
             .branch(isMultipleOfSeven);
   ```
   to be interpreted like 
   
   ```java
   BranchedKStream<Integer, String> branch = source.split()
            .branch(isEven, Branched.withFunction(Function.identity()))
            .branch(isMultipleOfSeven, Branched.withFunction(Function.identity()));
   ```
   
   Now back to Java: in Java, for `nothing` we have `null`. My original intention was to try to emulate `Function|Consumer` union type using overloading, but Java didn't allow me to do this dirty trick -- and separate `withFunction` and `withConsumer` builder methods appeared 😄 
   
   If I had succeded, it would have made sence to treat `null` uniformely for both functions and consumers. 
   
   Hope this explains why I allowed `null`s and always treated it as an identity function.
   
   But now I think you're right. I'm ready to **just prohibit passing in nulls and throw explicit NPEs for both functions and consumers**. This, at least, can spare us from further debates :-) What do you think?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548503824



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/Branched.java
##########
@@ -0,0 +1,150 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.function.Consumer;
+import java.util.function.Function;
+
+/**
+ * The {@code Branched} class is used to define the optional parameters when building branches with
+ * {@link BranchedKStream}.
+ *
+ * @param <K> type of record key
+ * @param <V> type of record value
+ */
+public class Branched<K, V> implements NamedOperation<Branched<K, V>> {
+
+    protected final String name;
+    protected final Function<? super KStream<K, V>,
+            ? extends KStream<K, V>> chainFunction;
+    protected final Consumer<? super KStream<K, V>> chainConsumer;
+
+    protected Branched(final String name,
+                       final Function<? super KStream<K, V>, ? extends KStream<K, V>> chainFunction,
+                       final Consumer<? super KStream<K, V>> chainConsumer) {
+        this.name = name;
+        this.chainFunction = chainFunction;
+        this.chainConsumer = chainConsumer;
+    }
+
+    /**
+     * Create an instance of {@link Branched} from an existing instance.
+     *
+     * @param branched the instance of {@link Branched} to copy
+     */
+    protected Branched(final Branched<K, V> branched) {
+        this(branched.name, branched.chainFunction, branched.chainConsumer);
+    }
+
+    /**
+     * Configure the instance of {@link Branched} with a branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null} a default branch name postfix will be generated (see
+     *             {@link BranchedKStream} description for details)
+     * @return {@code this}
+     */
+    @Override
+    public Branched<K, V> withName(final String name) {
+        return new Branched<>(name, chainFunction, chainConsumer);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided branch name postfix.
+     *
+     * @param name the branch name postfix to be used. If {@code null}, a default branch name postfix will be generated
+     *             (see {@link BranchedKStream} description for details)
+     * @param <K>  key type
+     * @param <V>  value type
+     * @return a new instance of {@link Branched}
+     */
+    public static <K, V> Branched<K, V> as(final String name) {
+        return new Branched<>(name, null, null);
+    }
+
+    /**
+     * Create an instance of {@link Branched} with provided chain function.
+     *
+     * @param chain A function that will be applied to the branch. If {@code null}, the identity
+     *              {@code kStream -> kStream} function will be supposed. If this function returns

Review comment:
       The KIP itself only says 'If no chain function or consumer is provided, then the value is the branch itself' which is implemented in `BranchedInternal` as following
   
   ```java
   if (chainFunction == null) {
               transformedStream = newStream;
           } else {
               transformedStream = chainFunction.apply(newStream);
           }
   ```
   
   So actually if `null` is passed in here, NPE won't happen, and JavaDoc just describes what actually happens. 
   
   But you might be right! I can't think of sanely reasons to explicitly pass in `null`s. But as a result of mistake, one can unintentionally pass `null`
   
   Shall I throw NPEs explicitly here and in similar places, WDYT?
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-750937120


   Hi @mjsax, thanks for your thorough revew! I have fixed everything according to your comments, except:
   
   * https://github.com/apache/kafka/pull/9107#discussion_r547592368 -- just want to be sure!
   * https://github.com/apache/kafka/pull/9107#discussion_r547606064 -- see my explanation
   * https://github.com/apache/kafka/pull/9107#discussion_r547607739 -- I agree that this is ugly but there was a good reason why I made it this way. Just give me a day and I'll return with comments (or will fix it).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: KIP-418 implementation

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-668302357


   I'll put it into my backlog. But I am the main reviewer for two other KIPs (216 and 466) that I should review first as they got approve earlier and PRs are open for longer already.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-764879234


   Thanks, @inponomarev !
   
   Yes, we sometimes have to update the KIP as the PR gets finalized. I like to send a quick reply to the vote thread on the mailing list to summarize the design changes, in case anyone wants to object (I don't think I've ever had someone object).


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-768478775


   @inponomarev the failing tests seems to be due to a known issue that was fixed via https://github.com/apache/kafka/pull/9768
   
   Can you rebase your PR to pickup the fix so we can get a green build?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-769038641


   Hi @mjsax , I have implemented all your suggestions concerning Javadocs/Scaladocs.
   
   After rebasing, there are less failing tests
   
   I also see that `JDK8` build continues to fail because of Scala compilation issue. AFAICS, that's because we are using `FunctionWrappers` object which is not available in older versions of Scala. Is it OK?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548519269



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).

Review comment:
       Rephrased: `if certain conditions are met, it also can be accessed from the {@link Map}`... -- and all these coditions are described below




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r565519912



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,169 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}. If a record does not match any predicates, it will be routed to the default branch,
+ * or dropped if no default branch is created.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. If certain conditions are met, it also can be accessed from the {@link Map} returned by
+ * {@link BranchedKStream#defaultBranch(Branched)} or {@link BranchedKStream#noDefaultBranch()}
+ * (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result

Review comment:
       Yes, I agree, unless you want to add a noun:
   
   ```suggestion
    * The branching happens on a first-match basis: A record in the original stream is assigned to the corresponding result
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-756855647


   Hey @inponomarev and @mjsax ! I'm glad to see this is moving along.
   
   Regarding https://github.com/apache/kafka/pull/9107#issuecomment-751922810 :
   
   My understanding was that defaultBranch/noDefaultBranch were the terminal operators, in that they close out the context of a BranchedKStream, and you can't add any more branches after one of those methods.
   
   But also, the whole branching construct is an incremental builder like the rest of the Kafka Streams API. In other words, just like this is a valid program:
   
   ```java
   builder.stream("input")
          .filter(myPredicate)
   ```
   
   so would be Ivan's example:
   
   ```java
   builder.split()
          .branch("myBranch", ...)
   ```
   
   What I mean by "incremental builder" is that each time you call a chained method in the DSL, it immediately adds nodes to the program, as opposed to having to call any kind of `build()` method to actually add stuff to the program. I think there are pros and cons to this design, but it seems more in line with the rest of the DSL not to require the terminal operators.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-762918424


   Thanks @inponomarev ,
   
   Ah, I didn't notice that method signature name. I actually prefer it this way :)
   
   Thanks also for pointing out the covariance change. This is also fine. Java's type system only contains a partial implementation of variance, so we do best we can.
   
   Did you already update the KIP? If not, please do.
   
   I'm +1 on this PR.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on a change in pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r564982061



##########
File path: streams/streams-scala/src/main/scala/org/apache/kafka/streams/scala/kstream/Branched.scala
##########
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.scala.kstream
+
+import org.apache.kafka.streams.kstream.{Branched => BranchedJ, KStream => KStreamJ}
+
+import scala.jdk.FunctionConverters._
+
+object Branched {
+
+  /**
+   * Create an instance of `Branched` with provided branch name suffix.
+   *
+   * @param name the branch name suffix to be used. If `null`, a default branch name suffix will be generated
+   *             (see [[BranchedKStream]] description for details)
+   * @tparam K key type
+   * @tparam V value type
+   * @return a new instance of `Branched`
+   */
+  def as[K, V](name: String): BranchedJ[K, V] =
+    BranchedJ.as[K, V](name)
+
+  /**
+   * Create an instance of `Branched` with provided chain function and branch name suffix.
+   *
+   * @param chain A function that will be applied to the branch. If `null`, the identity

Review comment:
       Remove `If null, the identity function will be used.` (we don't allow `null`)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-752382595


   As far as I can judge from the name, `@nowarn` is not for deprecation, but rather for a warning suppression  🤔 apparently we need to mirror the changes in Java `KStream` interface here. Never wrote anything in Scala before. OK, it's better to wait for @vvcephei !


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on a change in pull request #9107: KAFKA-5488: Add type save branch() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on a change in pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#discussion_r548521379



##########
File path: streams/src/main/java/org/apache/kafka/streams/kstream/BranchedKStream.java
##########
@@ -0,0 +1,172 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.kstream;
+
+import java.util.Map;
+
+/**
+ * Branches the records in the original stream based on the predicates supplied for the branch definitions.
+ * <p>
+ * Branches are defined with {@link BranchedKStream#branch(Predicate, Branched)} or
+ * {@link BranchedKStream#defaultBranch(Branched)} methods. Each record is evaluated against the predicates
+ * supplied via {@link Branched} parameters, and is routed to the first branch for which its respective predicate
+ * evaluates to {@code true}.
+ * <p>
+ * Each branch (which is a {@link KStream} instance) then can be processed either by
+ * a {@link java.util.function.Function} or a {@link java.util.function.Consumer} provided via a {@link Branched}
+ * parameter. It also can be accessed from the {@link Map} returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} (see <a href="#examples">usage examples</a>).
+ * <p>
+ * The branching happens on first-match: A record in the original stream is assigned to the corresponding result
+ * stream for the first predicate that evaluates to true, and is assigned to this stream only. If you need
+ * to route a record to multiple streams, you can use {@link KStream#filter(Predicate)} for each predicate instead
+ * of branching.
+ * <p>
+ * The process of routing the records to different branches is a stateless record-by-record operation.
+ * <h2><a name="maprules">Rules of forming the resulting map</a></h2>
+ * The keys of the {@code Map<String, KStream<K, V>>} entries returned by {@link BranchedKStream#defaultBranch(Branched)} or
+ * {@link BranchedKStream#noDefaultBranch()} are defined by the following rules:
+ * <p>
+ * <ul>
+ *     <li>If {@link Named} parameter was provided for {@link KStream#split(Named)}, its value is used as
+ *     a prefix for each key. By default, no prefix is used
+ *     <li>If a name is provided for the {@link BranchedKStream#branch(Predicate, Branched)} via
+ *     {@link Branched} parameter, its value is appended to the prefix to form the {@code Map} key

Review comment:
       Do we use dots at the end of list items? (Sorry -- English is my second language so I really don't know. I noticed that sometimes you do but in most cases you don't. In Russian language we always must use dots or commas at the end of list items, but Russian and English punctuation are completely different.)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] vvcephei commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
vvcephei commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-761683902


   Hey @inponomarev , I just took a look at the Scala API. Thanks for adding that!
   
   I figured it'd be just easier to push a few tweaks than to describe what needs to be done.
   
   * You asked me offline if we could avoid the overloads in Branched, and indeed, we can with a default argument of `null` for the name.
   * The Scala test was inadvertently using the Java Branched class, but you meant to test the Scala one.
   * I happened to notice a small typo: The opposite of "prefix" is "suffix", not "postfix"
   * I also noticed that your files all had CRLF (windows) return characters, so I fixed them. You might want to configure git for `autocrlf` (`git config --global core.autocrlf true`) (see https://www.git-scm.com/book/en/v2/Customizing-Git-Git-Configuration)
   
   These are all separate commits above, so you can scrutinize each one. This PR is your work, so feel free to protest any of my suggestions.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] inponomarev commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
inponomarev commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-768769254


   > @inponomarev the failing tests seems to be due to a known issue that was fixed via #9768
   > 
   > Can you rebase your PR to pickup the fix so we can get a green build?
   
   Done rebasing, expect the fixes according to your latest review soon!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [kafka] mjsax commented on pull request #9107: KAFKA-5488: Add type-safe split() operator

Posted by GitBox <gi...@apache.org>.
mjsax commented on pull request #9107:
URL: https://github.com/apache/kafka/pull/9107#issuecomment-773692994


   Merged to `trunk`.
   
   Congrats for getting this into the 2.8.0 release @inponomarev -- great work!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org