Class JoinedStreams<T1,T2>

java.lang.Object
org.apache.flink.streaming.api.datastream.JoinedStreams<T1,T2>

@Public public class JoinedStreams<T1,T2> extends Object
JoinedStreams represents two DataStreams that have been joined. A streaming join operation is evaluated over elements in a window.

To finalize the join operation you also need to specify a KeySelector for both the first and second input and a WindowAssigner.

Note: Right now, the join is being evaluated in memory so you need to ensure that the number of elements per key does not get too high. Otherwise the JVM might crash.

Example:


 DataStream<Tuple2<String, Integer>> one = ...;
 DataStream<Tuple2<String, Integer>> two = ...;

 DataStream<T> result = one.join(two)
     .where(new MyFirstKeySelector())
     .equalTo(new MyFirstKeySelector())
     .window(TumblingEventTimeWindows.of(Duration.ofSeconds(5)))
     .apply(new MyJoinFunction());
 
  • Nested Class Summary

    Nested Classes
    Modifier and Type
    Class
    Description
    class 
    Joined streams that have the key for one side defined.
    static class 
    A join operation that has KeySelectors defined for both inputs as well as a WindowAssigner.
  • Constructor Summary

    Constructors
    Constructor
    Description
    Creates new JoinedStreams data streams, which are the first step towards building a streaming co-group.
  • Method Summary

    Modifier and Type
    Method
    Description
    <KEY> JoinedStreams<T1,T2>.Where<KEY>
    where(org.apache.flink.api.java.functions.KeySelector<T1,KEY> keySelector)
    Specifies a KeySelector for elements from the first input.
    <KEY> JoinedStreams<T1,T2>.Where<KEY>
    where(org.apache.flink.api.java.functions.KeySelector<T1,KEY> keySelector, org.apache.flink.api.common.typeinfo.TypeInformation<KEY> keyType)
    Specifies a KeySelector for elements from the first input with explicit type information for the key type.

    Methods inherited from class java.lang.Object

    clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait
  • Constructor Details

    • JoinedStreams

      public JoinedStreams(DataStream<T1> input1, DataStream<T2> input2)
      Creates new JoinedStreams data streams, which are the first step towards building a streaming co-group.
      Parameters:
      input1 - The first data stream.
      input2 - The second data stream.
  • Method Details

    • where

      public <KEY> JoinedStreams<T1,T2>.Where<KEY> where(org.apache.flink.api.java.functions.KeySelector<T1,KEY> keySelector)
      Specifies a KeySelector for elements from the first input.
      Parameters:
      keySelector - The KeySelector to be used for extracting the key for partitioning.
    • where

      public <KEY> JoinedStreams<T1,T2>.Where<KEY> where(org.apache.flink.api.java.functions.KeySelector<T1,KEY> keySelector, org.apache.flink.api.common.typeinfo.TypeInformation<KEY> keyType)
      Specifies a KeySelector for elements from the first input with explicit type information for the key type.
      Parameters:
      keySelector - The KeySelector to be used for extracting the first input's key for partitioning.
      keyType - The type information describing the key type.