Class JoinWindows
 A JoinWindows instance defines a maximum time difference for a join over two streams on the same key.
 In SQL-style you would express this join as
 
     SELECT * FROM stream1, stream2
     WHERE
       stream1.key = stream2.key
       AND
       stream1.ts - before <= stream2.ts AND stream2.ts <= stream1.ts + after
 - before = after = time-difference
- before = 0 and after = time-difference
- before = time-difference and after = 0
Both values (before and after) must not result in an "inverse" window, i.e., upper-interval bound cannot be smaller than lower-interval bound.
 JoinWindows are sliding windows, thus, they are aligned to the actual record timestamps.
 This implies, that each input record defines its own window with start and end time being relative to the record's
 timestamp.
 
 For time semantics, see TimestampExtractor.
- See Also:
- 
- TimeWindows
- UnlimitedWindows
- SessionWindows
- KStream.join(KStream, ValueJoiner, JoinWindows)
- KStream.join(KStream, ValueJoiner, JoinWindows, StreamJoined)
- KStream.leftJoin(KStream, ValueJoiner, JoinWindows)
- KStream.leftJoin(KStream, ValueJoiner, JoinWindows, StreamJoined)
- KStream.outerJoin(KStream, ValueJoiner, JoinWindows)
- KStream.outerJoin(KStream, ValueJoiner, JoinWindows, StreamJoined)
- TimestampExtractor
 
- 
Field SummaryFieldsModifier and TypeFieldDescriptionfinal longMaximum time difference for tuples that are after the join tuple.final longMaximum time difference for tuples that are before the join tuple.protected final booleanEnable left/outer stream-stream join, by not emitting left/outer results eagerly, but only after the grace period passed.Fields inherited from class org.apache.kafka.streams.kstream.WindowsDEPRECATED_DEFAULT_24_HR_GRACE_PERIOD, NO_GRACE_PERIOD
- 
Constructor SummaryConstructors
- 
Method SummaryModifier and TypeMethodDescriptionChanges the end window boundary totimeDifferencebut keep the start window boundary as is.Changes the start window boundary totimeDifferencebut keep the end window boundary as is.booleanDeprecated.since 3.0.longReturn the window grace period (the time to admit out-of-order events after the end of the window.) Delay is defined as (stream_time - record_timestamp).inthashCode()static JoinWindowsDeprecated.since 3.0.static JoinWindowsofTimeDifferenceAndGrace(Duration timeDifference, Duration afterWindowEnd) Specifies that records of the same key are joinable if their timestamps are withintimeDifference, i.e., the timestamp of a record from the secondary stream is maxtimeDifferencebefore or after the timestamp of the record from the primary stream.static JoinWindowsofTimeDifferenceWithNoGrace(Duration timeDifference) Specifies that records of the same key are joinable if their timestamps are withintimeDifference, i.e., the timestamp of a record from the secondary stream is maxtimeDifferencebefore or after the timestamp of the record from the primary stream.longsize()Return the size of the specified windows in milliseconds.toString()windowsFor(long timestamp) Not supported byJoinWindows.
- 
Field Details- 
beforeMspublic final long beforeMsMaximum time difference for tuples that are before the join tuple.
- 
afterMspublic final long afterMsMaximum time difference for tuples that are after the join tuple.
- 
enableSpuriousResultFixprotected final boolean enableSpuriousResultFixEnable left/outer stream-stream join, by not emitting left/outer results eagerly, but only after the grace period passed. This flag can only be enabled via ofTimeDifferenceAndGrace or ofTimeDifferenceWithNoGrace.
 
- 
- 
Constructor Details- 
JoinWindows
 
- 
- 
Method Details- 
ofTimeDifferenceAndGracepublic static JoinWindows ofTimeDifferenceAndGrace(Duration timeDifference, Duration afterWindowEnd) Specifies that records of the same key are joinable if their timestamps are withintimeDifference, i.e., the timestamp of a record from the secondary stream is maxtimeDifferencebefore or after the timestamp of the record from the primary stream.Using this method explicitly sets the grace period to the duration specified by afterWindowEnd, which means that only out-of-order records arriving more than the grace period after the window end will be dropped. The window close, after which any incoming records are considered late and will be rejected, is defined aswindowEnd + afterWindowEnd- Parameters:
- timeDifference- join window interval
- afterWindowEnd- The grace period to admit out-of-order events to a window.
- Returns:
- A new JoinWindows object with the specified window definition and grace period
- Throws:
- IllegalArgumentException- if- timeDifferenceis negative or can't be represented as- long millisecondsif- afterWindowEndis negative or can't be represented as- long milliseconds
 
- 
ofTimeDifferenceWithNoGraceSpecifies that records of the same key are joinable if their timestamps are withintimeDifference, i.e., the timestamp of a record from the secondary stream is maxtimeDifferencebefore or after the timestamp of the record from the primary stream.CAUTION: Using this method implicitly sets the grace period to zero, which means that any out-of-order records arriving after the window ends are considered late and will be dropped. - Parameters:
- timeDifference- join window interval
- Returns:
- a new JoinWindows object with the window definition and no grace period. Note that this means out-of-order records arriving after the window end will be dropped
- Throws:
- IllegalArgumentException- if- timeDifferenceis negative or can't be represented as- long milliseconds
 
- 
ofDeprecated.since 3.0. UseofTimeDifferenceWithNoGrace(Duration)} insteadSpecifies that records of the same key are joinable if their timestamps are withintimeDifference, i.e., the timestamp of a record from the secondary stream is maxtimeDifferencebefore or after the timestamp of the record from the primary stream.- Parameters:
- timeDifference- join window interval
- Returns:
- a new JoinWindows object with the window definition with and grace period (default to 24 hours minus timeDifference)
- Throws:
- IllegalArgumentException- if- timeDifferenceis negative or can't be represented as- long milliseconds
 
- 
beforeChanges the start window boundary totimeDifferencebut keep the end window boundary as is. Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at mosttimeDifferenceearlier than the timestamp of the record from the primary stream.timeDifferencecan be negative but its absolute value must not be larger than current window "after" value (which would result in a negative window size).- Parameters:
- timeDifference- relative window start time
- Throws:
- IllegalArgumentException- if the resulting window size is negative or- timeDifferencecan't be represented as- long milliseconds
 
- 
afterChanges the end window boundary totimeDifferencebut keep the start window boundary as is. Thus, records of the same key are joinable if the timestamp of a record from the secondary stream is at mosttimeDifferencelater than the timestamp of the record from the primary stream.timeDifferencecan be negative but its absolute value must not be larger than current window "before" value (which would result in a negative window size).- Parameters:
- timeDifference- relative window end time
- Throws:
- IllegalArgumentException- if the resulting window size is negative or- timeDifferencecan't be represented as- long milliseconds
 
- 
windowsForNot supported byJoinWindows. ThrowsUnsupportedOperationException.- Specified by:
- windowsForin class- Windows<Window>
- Parameters:
- timestamp- the timestamp window should get created for
- Returns:
- a map of windowStartTimestamp -> Windowentries
- Throws:
- UnsupportedOperationException- at every invocation
 
- 
sizepublic long size()Description copied from class:WindowsReturn the size of the specified windows in milliseconds.
- 
graceDeprecated.since 3.0. UseofTimeDifferenceAndGrace(Duration, Duration)insteadReject out-of-order events that are delayed more thanafterWindowEndafter the end of its window.Delay is defined as (stream_time - record_timestamp). - Parameters:
- afterWindowEnd- The grace period to admit out-of-order events to a window.
- Returns:
- this updated builder
- Throws:
- IllegalArgumentException- if the- afterWindowEndis negative or can't be represented as- long milliseconds
- IllegalStateException- if- grace(Duration)is called after- ofTimeDifferenceAndGrace(Duration, Duration)or- ofTimeDifferenceWithNoGrace(Duration)
 
- 
gracePeriodMspublic long gracePeriodMs()Description copied from class:WindowsReturn the window grace period (the time to admit out-of-order events after the end of the window.) Delay is defined as (stream_time - record_timestamp).- Specified by:
- gracePeriodMsin class- Windows<Window>
 
- 
equals
- 
hashCodepublic int hashCode()
- 
toString
 
-