Interface WatermarkStrategyWithPunctuatedWatermarks<T>

  • Type Parameters:
    T - The type of the elements to which this assigner assigns timestamps.
    All Superinterfaces:
    Serializable, org.apache.flink.api.common.eventtime.TimestampAssigner<T>, org.apache.flink.api.common.eventtime.TimestampAssignerSupplier<T>, org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier<T>, org.apache.flink.api.common.eventtime.WatermarkStrategy<T>

    @Internal
    public interface WatermarkStrategyWithPunctuatedWatermarks<T>
    extends org.apache.flink.api.common.eventtime.WatermarkStrategy<T>, org.apache.flink.api.common.eventtime.TimestampAssigner<T>
    The AssignerWithPunctuatedWatermarks assigns event time timestamps to elements, and generates low watermarks that signal event time progress within the stream. These timestamps and watermarks are used by functions and operators that operate on event time, for example event time windows.

    Use this class if certain special elements act as markers that signify event time progress, and when you want to emit watermarks specifically at certain events. The system will generate a new watermark, if the probed value is non-null and has a timestamp larger than that of the previous watermark (to preserve the contract of ascending watermarks).

    For use cases that should periodically emit watermarks based on element timestamps, use the WatermarkStrategyWithPeriodicWatermarks instead.

    The following example illustrates how to use this timestamp extractor and watermark generator. It assumes elements carry a timestamp that describes when they were created, and that some elements carry a flag, marking them as the end of a sequence such that no elements with smaller timestamps can come anymore.

    
     public class WatermarkOnFlagAssigner implements AssignerWithPunctuatedWatermarks<MyElement> {
    
         public long extractTimestamp(MyElement element, long previousElementTimestamp) {
             return element.getSequenceTimestamp();
         }
    
         public Watermark checkAndGetNextWatermark(MyElement lastElement, long extractedTimestamp) {
             return lastElement.isEndOfSequence() ? new Watermark(extractedTimestamp) : null;
         }
     }
     

    Timestamps and watermarks are defined as longs that represent the milliseconds since the Epoch (midnight, January 1, 1970 UTC). A watermark with a certain value t indicates that no elements with event timestamps x, where x is lower or equal to t, will occur any more.

    See Also:
    Watermark
    • Nested Class Summary

      • Nested classes/interfaces inherited from interface org.apache.flink.api.common.eventtime.TimestampAssignerSupplier

        org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.Context, org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.SupplierFromSerializableTimestampAssigner<T extends Object>
      • Nested classes/interfaces inherited from interface org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier

        org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context
    • Field Summary

      • Fields inherited from interface org.apache.flink.api.common.eventtime.TimestampAssigner

        NO_TIMESTAMP
    • Method Summary

      All Methods Instance Methods Abstract Methods Default Methods 
      Modifier and Type Method Description
      Watermark checkAndGetNextWatermark​(T lastElement, long extractedTimestamp)
      Asks this implementation if it wants to emit a watermark.
      default org.apache.flink.api.common.eventtime.TimestampAssigner<T> createTimestampAssigner​(org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.Context context)  
      default org.apache.flink.api.common.eventtime.WatermarkGenerator<T> createWatermarkGenerator​(org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context context)  
      • Methods inherited from interface org.apache.flink.api.common.eventtime.TimestampAssigner

        extractTimestamp
      • Methods inherited from interface org.apache.flink.api.common.eventtime.WatermarkStrategy

        getAlignmentParameters, withIdleness, withTimestampAssigner, withTimestampAssigner, withWatermarkAlignment, withWatermarkAlignment
    • Method Detail

      • checkAndGetNextWatermark

        @Nullable
        Watermark checkAndGetNextWatermark​(T lastElement,
                                           long extractedTimestamp)
        Asks this implementation if it wants to emit a watermark. This method is called right after the TimestampAssigner.extractTimestamp(Object, long) method.

        The returned watermark will be emitted only if it is non-null and its timestamp is larger than that of the previously emitted watermark (to preserve the contract of ascending watermarks). If a null value is returned, or the timestamp of the returned watermark is smaller than that of the last emitted one, then no new watermark will be generated.

        For an example how to use this method, see the documentation of this class.

        Returns:
        Null, if no watermark should be emitted, or the next watermark to emit.
      • createTimestampAssigner

        default org.apache.flink.api.common.eventtime.TimestampAssigner<T> createTimestampAssigner​(org.apache.flink.api.common.eventtime.TimestampAssignerSupplier.Context context)
        Specified by:
        createTimestampAssigner in interface org.apache.flink.api.common.eventtime.TimestampAssignerSupplier<T>
        Specified by:
        createTimestampAssigner in interface org.apache.flink.api.common.eventtime.WatermarkStrategy<T>
      • createWatermarkGenerator

        default org.apache.flink.api.common.eventtime.WatermarkGenerator<T> createWatermarkGenerator​(org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier.Context context)
        Specified by:
        createWatermarkGenerator in interface org.apache.flink.api.common.eventtime.WatermarkGeneratorSupplier<T>
        Specified by:
        createWatermarkGenerator in interface org.apache.flink.api.common.eventtime.WatermarkStrategy<T>