前往小程序,Get更优阅读体验!
立即前往
首页
学习
活动
专区
工具
TVP
发布
社区首页 >专栏 >flink watermark介绍

flink watermark介绍

作者头像
sanmutongzi
发布2020-03-04 15:38:53
7490
发布2020-03-04 15:38:53
举报
文章被收录于专栏:stream processstream process

转发请注明原创地址 http://www.cnblogs.com/dongxiao-yang/p/7610412.html

一 概念

watermark是flink为了处理eventTime窗口计算提出的一种机制,本质上也是一种时间戳,由flink souce或者自定义的watermark生成器按照需求定期或者按条件生成一种系统event,与普通数据流event一样流转到对应的下游operations,接收到watermark数据的operator以此不断调整自己管理的window event time clock。

( A watermark is a special event signaling that time in the event stream (i.e., the real-world timestamps in the event stream) has reached a certain point (say, 10am), and thus no event with timestamp earlier than 10am will arrive from now on. These watermarks are part of the data stream alongside regular events, and a Flink operator advances its event time clock to 10am once it has received a 10am watermark from all its upstream operations/sources)

二 TimestampAssigner和Watermark

首先,eventTime计算意味着flink必须有一个地方用于抽取每条消息中自带的时间戳,所以TimestampAssigner的实现类都要具体实现

long extractTimestamp(T element, long previousElementTimestamp);方法用来抽取当前元素的eventTime,这个eventTime会用来决定元素落到下游的哪个或者哪几个window中进行计算。

其次,在数据进入window前,需要有一个Watermarker生成当前的event time对应的水位线,flink支持两种后置的Watermarker:Periodic和Punctuated,一种是定期产生watermark(即使没有消息产生),一种是在满足特定情况的前提下触发。两种Watermark分别需要实现接口为

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

Watermark getCurrentWatermark()和Watermark checkAndGetNextWatermark(T lastElement, long extractedTimestamp);

帖几个官网给出的实现样例

Periodic Watermarks
/**
 * This generator generates watermarks assuming that elements arrive out of order,
 * but only to a certain degree. The latest elements for a certain timestamp t will arrive
 * at most n milliseconds after the earliest elements for timestamp t.
 */
public class BoundedOutOfOrdernessGenerator extends AssignerWithPeriodicWatermarks<MyEvent> {

    private final long maxOutOfOrderness = 3500; // 3.5 seconds

    private long currentMaxTimestamp;

    @Override
    public long extractTimestamp(MyEvent element, long previousElementTimestamp) {
        long timestamp = element.getCreationTime();
        currentMaxTimestamp = Math.max(timestamp, currentMaxTimestamp);
        return timestamp;
    }

    @Override
    public Watermark getCurrentWatermark() {
        // return the watermark as current highest timestamp minus the out-of-orderness bound
        return new Watermark(currentMaxTimestamp - maxOutOfOrderness);
    }
}

/**
 * This generator generates watermarks that are lagging behind processing time by a fixed amount.
 * It assumes that elements arrive in Flink after a bounded delay.
 */
public class TimeLagWatermarkGenerator extends AssignerWithPeriodicWatermarks<MyEvent> {

    private final long maxTimeLag = 5000; // 5 seconds

    @Override
    public long extractTimestamp(MyEvent element, long previousElementTimestamp) {
        return element.getCreationTime();
    }

    @Override
    public Watermark getCurrentWatermark() {
        // return the watermark as current time minus the maximum time lag
        return new Watermark(System.currentTimeMillis() - maxTimeLag);
    }
}
Punctuated Watermarks
public class PunctuatedAssigner extends AssignerWithPunctuatedWatermarks<MyEvent> {

    @Override
    public long extractTimestamp(MyEvent element, long previousElementTimestamp) {
        return element.getCreationTime();
    }

    @Override
    public Watermark checkAndGetNextWatermark(MyEvent lastElement, long extractedTimestamp) {
        return lastElement.hasWatermarkMarker() ? new Watermark(extractedTimestamp) : null;
    }
}

三代码调试

public class WindowWaterMark {

    public static void main(String[] args) throws Exception {
        // TODO Auto-generated method stub


        String hostName = "localhost";
        Integer port = Integer.parseInt("8001");

        // set up the execution environment
        final StreamExecutionEnvironment env = StreamExecutionEnvironment
                .getExecutionEnvironment();
        env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
        env.setParallelism(2);
         env.getConfig().setAutoWatermarkInterval(9000);

        // get input data
        DataStream<String> text = env.socketTextStream(hostName, port);

        DataStream<Tuple3<String, Long, Integer>> counts =
    
        text.filter(new FilterClass()).map(new LineSplitter())
        .assignTimestampsAndWatermarks(new AssignerWithPeriodicWatermarks<Tuple3<String, Long, Integer>>(){

            private long  currentMaxTimestamp = 0l;
            private final long maxOutOfOrderness = 10000l;
            
            @Override
            public long extractTimestamp(Tuple3<String, Long, Integer> element,
                    long previousElementTimestamp) {
                // TODO Auto-generated method stub
                long timestamp= element.f1;
                currentMaxTimestamp = Math.max(timestamp, currentMaxTimestamp);
                System.out.println("get timestamp is "+timestamp+" currentMaxTimestamp "+currentMaxTimestamp);
                return timestamp;
            }

            @Override
            public Watermark getCurrentWatermark() {
                // TODO Auto-generated method stub
                System.out.println("wall clock is "+ System.currentTimeMillis() +" new watermark "+(currentMaxTimestamp - maxOutOfOrderness));
                   return new Watermark(currentMaxTimestamp - maxOutOfOrderness);
                    
            }
            
        })
        .keyBy(0)
        .timeWindow(Time.seconds(20))
//        .allowedLateness(Time.seconds(10))
        .sum(2);

        counts.print();

        // execute program
        env.execute("Java WordCount from SocketTextStream Example");
    }


    public static final class LineSplitter implements
            MapFunction<String, Tuple3<String, Long, Integer>> {

        @Override
        public Tuple3<String, Long, Integer> map(String value) throws Exception {
            // TODO Auto-generated method stub
            String[] tokens = value.toLowerCase().split("\\W+");

            long eventtime = Long.parseLong(tokens[1]);
            
            return new Tuple3<String, Long, Integer>(tokens[0], eventtime, 1);
        }
    }

    private static class MyTimestamp extends
            AscendingTimestampExtractor<Tuple3<String, Long, Integer>> {
        private static final long serialVersionUID = 1L;

        @Override
        public long extractAscendingTimestamp(
                Tuple3<String, Long, Integer> element) {
            // TODO Auto-generated method stub
            return element.f1;
        }

    }
    
    
    public static final class FilterClass implements FilterFunction<String>
    {

        @Override
        public boolean filter(String value) throws Exception {
            // TODO Auto-generated method stub
            if(StringUtils.isNullOrWhitespaceOnly(value))
            {
                return false;
            }
            else
            {
                return true;
            }
        }
        
    }
    
}

测试代码如上,注意这段代码手动更改了autowatermarkinterval的时间为9s以便于观察方法调用顺序。

首先启动job不输入数据,30s后日志输出为

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

wall clock is 1506680562679 new watermark -10000

wall clock is 1506680562679 new watermark -10000

wall clock is 1506680571683 new watermark -10000

wall clock is 1506680571683 new watermark -10000

wall clock is 1506680580687 new watermark -10000

wall clock is 1506680580687 new watermark -10000

.........................

这说明在没有数据输入的情况下PeriodicWatermarks仍然会周期性调用getCurrentWatermark这个方法,每次有两条相同wall clock的日志跟程序里env.setParallelism(2)这个参数相同,表明watermark与operator的并发一致。

输入数据aaaa 1506590035000

日志输出为

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

wall clock is 1506681868124 new watermark -10000

wall clock is 1506681877129 new watermark -10000

wall clock is 1506681877129 new watermark -10000

get timestamp is 1506590035000 currentMaxTimestamp 1506590035000

wall clock is 1506681886132 new watermark 1506590025000

wall clock is 1506681886132 new watermark -10000

wall clock is 1506681895136 new watermark -10000

wall clock is 1506681895136 new watermark 1506590025000

...........................................

上述日志表明接收到消息后extractTimestamp这个方法会被立即调用,但是同时注意到wall clock日志的打印时间完全没有受到数据流入的影响,所以在PeriodicWatermarks这个是线下,watermark的产生时间和速率与数据流的输入无关。

需要说明的是,时间窗口的起始时间计算方法为

public static long getWindowStartWithOffset(long timestamp, long offset, long windowSize) {
   return timestamp - (timestamp - offset + windowSize) % windowSize;
}

所有对于上述测试代码里时间长度为20s的滚动窗口,默认下,在每分钟内窗口的起止时间都是(0~20)(20~40)(40~60)这样,我们的第一条数据aaaa 1506590035000对应时间为2017/9/28 17:13:55,所以它将会在(2017/9/28 17:13:40~2017/9/28 17:14:00)这个窗口完成计算

继续输入数据

cc 1506590035000 cc 1506590035000 bb 1506590035000 aaaa 1506590035000 bb 1506590035000

aaaa 1506590041000 //上调数据的eventTime至2017/9/28 17:14:01,超过前一个window的结束时间 bb 1506590041000 cc 1506590041000

日志输出为

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

get timestamp is 1506590041000 currentMaxTimestamp 1506590041000

wall clock is 1507522499419 new watermark 1506590031000

wall clock is 1507522499424 new watermark 1506590025000

wall clock is 1507522508422 new watermark 1506590031000

wall clock is 1507522508429 new watermark 1506590025000

wall clock is 1507522517426 new watermark 1506590031000

wall clock is 1507522517434 new watermark 1506590025000

get timestamp is 1506590041000 currentMaxTimestamp 1506590041000

wall clock is 1507522526429 new watermark 1506590031000

wall clock is 1507522526435 new watermark 1506590031000

wall clock is 1507522535431 new watermark 1506590031000

wall clock is 1507522535440 new watermark 1506590031000

wall clock is 1507522544433 new watermark 1506590031000

可以看到虽然后来的数据已经超过了第一个window的endtime,但是由于getCurrentWatermark方法的设定系统目前的watermark为2017/9/28 17:13:51小于endtime,所以flink并不会立即执行整个窗口的运算

继续增加数据和eventtime

aaaa 1506590051000 bb 1506590051000 cc 1506590051000

日志输出如下

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

get timestamp is 1506590051000 currentMaxTimestamp 1506590051000

get timestamp is 1506590051000 currentMaxTimestamp 1506590051000

get timestamp is 1506590051000 currentMaxTimestamp 1506590051000

wall clock is 1507522589449 new watermark 1506590041000

wall clock is 1507522589461 new watermark 1506590041000

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

1> (aaaa,1506590035000,2)

2> (cc,1506590035000,2)

2> (bb,1506590035000,2)

这个时候watermark刚好大于了第一个window的endtime,整个(2017/9/28 17:13:40~2017/9/28 17:14:00)窗口对应的数据开始执行计算,输出对应结果。

参考文档

1 http://vishnuviswanath.com/flink_eventtime.html

2 https://data-artisans.com/blog/how-apache-flink-enables-new-streaming-applications-part-1

3 https://www.youtube.com/watch?v=3UfZN59Nsk8

4 Flink流计算编程--watermark(水位线)简介

<!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} span.s1 {color: #931a68} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} span.s1 {color: #931a68} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} span.s1 {color: #7e504f} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} --><!-- p.p1 {margin: 0.0px 0.0px 0.0px 0.0px; font: 11.0px Monaco} -->

本文参与 腾讯云自媒体分享计划,分享自作者个人站点/博客。
原始发表:2017-10-09 ,如有侵权请联系 cloudcommunity@tencent.com 删除

本文分享自 作者个人站点/博客 前往查看

如有侵权,请联系 cloudcommunity@tencent.com 删除。

本文参与 腾讯云自媒体分享计划  ,欢迎热爱写作的你一起参与!

评论
登录后参与评论
0 条评论
热度
最新
推荐阅读
目录
  • Periodic Watermarks
  • Punctuated Watermarks
相关产品与服务
大数据
全栈大数据产品,面向海量数据场景,帮助您 “智理无数,心中有数”!
领券
问题归档专栏文章快讯文章归档关键词归档开发者手册归档开发者手册 Section 归档