博客
关于我
强烈建议你试试无所不能的chatGPT,快点击我
基于flink broadcast实现目标用户的智能筛选
阅读量:3959 次
发布时间:2019-05-24

本文共 17036 字,大约阅读时间需要 56 分钟。

1、broadcast简介

支持Operator State的第三种类型是广播状态,其中需要将来自一个流的某些或者全部数据广播到所有下游任务,广播的状态将存储在本地,用于处理另一个流上所有传入的元素。

2、API使用

博主分享两个案例分别描述key_broadcast和nonKeyed_broadcast。

需求描述:某电商系统,需要通过监控用户的浏览记录,根据不同商品的营销活动规则,筛选出目标用户,实现促销。

案例一: nonKeyed_broadcast

(1) 实体类准备

package com.learn.noKeyedBroadcast;/** * 描述:商品类别的阈值 */public class Rule {    private String channel;   //商品类别 如:家电    private Integer threshold;//达到浏览路径长度的阈值 如:该类商品浏览了5次    public Rule(String channel, Integer threshold) {        this.channel = channel;        this.threshold = threshold;    }    public String getChannel() {        return channel;    }    public void setChannel(String channel) {        this.channel = channel;    }    public Integer getThreshold() {        return threshold;    }    public void setThreshold(Integer threshold) {        this.threshold = threshold;    }    @Override    public String toString() {        return "Rule{" +                "channel='" + channel + '\'' +                ", threshold=" + threshold +                '}';    }}
package com.learn.noKeyedBroadcast;import java.io.Serializable;/** * 描述:用户浏览商城产生的日志 */public class UserAction implements Serializable {    private String id;       //商品编号 如:001    private String name;     //商品名称 如:电视机    private String channel;  //商品类别 如:家电    private String action;   //用户行为 如:购买、添加购物车、浏览    public UserAction(String id, String name, String channel, String action) {        this.id = id;        this.name = name;        this.channel = channel;        this.action = action;    }    public String getId() {        return id;    }    public void setId(String id) {        this.id = id;    }    public String getName() {        return name;    }    public void setName(String name) {        this.name = name;    }    public String getChannel() {        return channel;    }    public void setChannel(String channel) {        this.channel = channel;    }    public String getAction() {        return action;    }    public void setAction(String action) {        this.action = action;    }    @Override    public String toString() {        return "UserAction{" +                "id='" + id + '\'' +                ", name='" + name + '\'' +                ", channel='" + channel + '\'' +                ", action='" + action + '\'' +                '}';    }}
package com.learn.noKeyedBroadcast;import java.io.Serializable;/** * 描述:对用户的浏览路径进行分类汇总的载体 */public class UserBuyPath implements Serializable {    private String id;    private String name;    private String channel;    private Integer path;   //浏览同类别商品的路径长度    public UserBuyPath(String id, String name, String channel, Integer path) {        this.id = id;        this.name = name;        this.channel = channel;        this.path = path;    }    public String getId() {        return id;    }    public void setId(String id) {        this.id = id;    }    public String getName() {        return name;    }    public void setName(String name) {        this.name = name;    }    public String getChannel() {        return channel;    }    public void setChannel(String channel) {        this.channel = channel;    }    public Integer getPath() {        return path;    }    public void setPath(Integer path) {        this.path = path;    }    @Override    public String toString() {        return "UserBuyPath{" +                "id='" + id + '\'' +                ", name='" + name + '\'' +                ", channel='" + channel + '\'' +                ", path=" + path +                '}';    }}

(2)API准备

package com.learn.noKeyedBroadcast;import org.apache.flink.api.common.functions.RichMapFunction;import org.apache.flink.api.common.state.MapState;import org.apache.flink.api.common.state.MapStateDescriptor;import org.apache.flink.api.common.typeinfo.TypeHint;import org.apache.flink.api.common.typeinfo.TypeInformation;import org.apache.flink.configuration.Configuration;/** * 描述:对用户行为进行统计,结果存储在state中,如果用户发生了购买行为,则将当前用户的状态清除 */public class UserActionRichMapFunction extends RichMapFunction
{ private transient MapState
buyPathState; @Override public void open(Configuration parameters) throws Exception { super.open(parameters); MapStateDescriptor
descriptor = new MapStateDescriptor<>( "buy_path" , TypeInformation.of(new TypeHint
() {}), TypeInformation.of(new TypeHint
() {}) ); buyPathState = getRuntimeContext().getMapState(descriptor); } @Override public UserBuyPath map(UserAction value) throws Exception { String channel = value.getChannel(); Integer path = 0; if(buyPathState.contains(channel)){ path = buyPathState.get(channel); } if(value.getAction().equals("buy")){ buyPathState.remove(channel); }else{ buyPathState.put(channel,path+1); } return new UserBuyPath(value.getId(),value.getName(),value.getChannel(),buyPathState.get(channel)); }}
package com.learn.noKeyedBroadcast;import org.apache.flink.api.common.state.*;import org.apache.flink.streaming.api.functions.co.BroadcastProcessFunction;import org.apache.flink.util.Collector;import java.util.Map;/** * 描述:通过print的方式将活动规则展示,交互用户行为数据,符合条件的数据通过out输出 */public class UserBuyPathBroadcastProcessFunction extends BroadcastProcessFunction
{ private MapStateDescriptor
mapState; public UserBuyPathBroadcastProcessFunction(MapStateDescriptor
mapState) { this.mapState = mapState; } //处理的是UserBuyPath,读取广播状态 @Override public void processElement(UserBuyPath value, ReadOnlyContext ctx, Collector
out) throws Exception { ReadOnlyBroadcastState
broadcastState = ctx.getBroadcastState(mapState); if(broadcastState.contains(value.getChannel())){//如果有规则,尝试计算 Integer threshold = broadcastState.get(value.getChannel()); if(value.getPath() >= threshold){ //将满足条件的用户信息输出 out.collect(value.getId()+" "+value.getName()+" "+value.getChannel()+" "+value.getPath()); } } } //处理的是规则 Rule 数据 ,记录修改广播状态 @Override public void processBroadcastElement(Rule value, Context ctx, Collector
out) throws Exception { BroadcastState
broadcastState = ctx.getBroadcastState(mapState); broadcastState.put(value.getChannel(),value.getThreshold()); System.out.println("=======当前商家活动规则规则如下======"); Iterable
> entries = broadcastState.entries(); for (Map.Entry
entry : entries) { System.out.println(entry.getKey()+"\t"+entry.getValue()); } }}

(3)flink主类

package com.learn.noKeyedBroadcast;import org.apache.flink.api.common.serialization.SimpleStringSchema;import org.apache.flink.api.common.state.MapStateDescriptor;import org.apache.flink.api.common.state.StateTtlConfig;import org.apache.flink.api.common.time.Time;import org.apache.flink.api.common.typeinfo.TypeHint;import org.apache.flink.api.common.typeinfo.TypeInformation;import org.apache.flink.streaming.api.datastream.BroadcastStream;import org.apache.flink.streaming.api.datastream.DataStreamSource;import org.apache.flink.streaming.api.datastream.SingleOutputStreamOperator;import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011;import java.util.Properties;public class Driver {    public static void main(String[] args) throws Exception {        //1、flink运行环境        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();        env.setParallelism(1); //方便效果展示,全局并行度设置1        //2.1 模拟用户访问商城产生的日志        /* id name channel action           001 mack 手机 view           001 dell 手机 addToCart        * */        Properties properties = new Properties();        properties.setProperty("bootstrap.servers","centos:9092");        properties.setProperty("group.id", "aa");        FlinkKafkaConsumer011
kafkaSource0 = new FlinkKafkaConsumer011<>("aaaa", new SimpleStringSchema(), properties); DataStreamSource
kafkaSource01 = env.addSource(kafkaSource0); //2.2 模拟商家发放的优惠券或者促销活动 /* channel 阈值 手机 5 服装 3 */ Properties properties02 = new Properties(); properties02.setProperty("bootstrap.servers","centos:9092"); properties02.setProperty("group.id", "aa"); FlinkKafkaConsumer011
kafkaSource1 = new FlinkKafkaConsumer011<>("bbbb", new SimpleStringSchema(), properties02); DataStreamSource
kafkaSource02 = env.addSource(kafkaSource1); //3、用户访问生成的日志做统计 SingleOutputStreamOperator
useStream = kafkaSource01 .map(t -> t.split(" ")) .map(t -> new UserAction(t[0], t[1], t[2], t[3])) .keyBy(t -> t.getId() + t.getName()) .map(new UserActionRichMapFunction()); //4、mapState作为商家活动规则的载体,模拟优惠券或者活动的时间 MapStateDescriptor
mapState = new MapStateDescriptor<>("braodcast-sate", TypeInformation.of(new TypeHint
() { }), TypeInformation.of(new TypeHint
() { }) ); StateTtlConfig ttlConfig = StateTtlConfig.newBuilder(Time.seconds(1)) .setUpdateType(StateTtlConfig.UpdateType.OnCreateAndWrite) .setStateVisibility(StateTtlConfig.StateVisibility.NeverReturnExpired) .build(); mapState.enableTimeToLive(ttlConfig); //5、将商家的活动规则进行广播 BroadcastStream
broadcastStream = kafkaSource02 .map(t -> t.split(" ")) .map(t -> new Rule(t[0], Integer.parseInt(t[1]))) .broadcast(mapState); //6、数据处理 useStream.connect(broadcastStream) .process(new UserBuyPathBroadcastProcessFunction(mapState)) .printToErr(); //筛选出的目标用户用打印模拟 env.execute("metricsCounter"); }}

案例二: key_broadcast

package com.learn.keyedBroadcast;import com.learn.noKeyedBroadcast.Rule;import com.learn.noKeyedBroadcast.UserAction;import com.learn.noKeyedBroadcast.*;import org.apache.flink.api.common.serialization.SimpleStringSchema;import org.apache.flink.api.common.state.*;import org.apache.flink.api.common.time.Time;import org.apache.flink.api.common.typeinfo.BasicTypeInfo;import org.apache.flink.api.common.typeinfo.TypeHint;import org.apache.flink.api.common.typeinfo.TypeInformation;import org.apache.flink.api.java.typeutils.MapTypeInfo;import org.apache.flink.configuration.Configuration;import org.apache.flink.streaming.api.datastream.*;import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;import org.apache.flink.streaming.api.functions.co.KeyedBroadcastProcessFunction;import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer011;import org.apache.flink.util.Collector;import java.util.Iterator;import java.util.Map;import java.util.Properties;public class Driver {    public static void main(String[] args) throws Exception {        //1、flink运行环境        StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();        env.setParallelism(1);        //2.1 模拟用户访问商城产生的日志        /* id name channel action           001 mack 手机 view        * */        Properties properties = new Properties();        properties.setProperty("bootstrap.servers","centos:9092");        properties.setProperty("group.id", "aa");        FlinkKafkaConsumer011
kafkaSource0 = new FlinkKafkaConsumer011<>("aaaa", new SimpleStringSchema(), properties); kafkaSource0.setStartFromLatest(); DataStreamSource
kafkaSource01 = env.addSource(kafkaSource0); //2.2 模拟商家发放的优惠券或者促销活动 /* channel 阈值 手机 5 */ Properties properties02 = new Properties(); properties02.setProperty("bootstrap.servers","centos:9092"); properties02.setProperty("group.id", "aa"); FlinkKafkaConsumer011
kafkaSource1 = new FlinkKafkaConsumer011<>("bbbb", new SimpleStringSchema(), properties02); kafkaSource1.setStartFromLatest(); DataStreamSource
kafkaSource02 = env.addSource(kafkaSource1); //3、用户访问生成的日志做统计 KeyedStream
useStream = kafkaSource01 .map(t -> t.split(" ")) .map(t -> new UserAction(t[0], t[1], t[2], t[3])) .keyBy(t -> t.getId()); //4、mapState作为商家活动规则的载体,模拟优惠券或者活动的时间 MapStateDescriptor
mapState = new MapStateDescriptor<>("braodcast-sate", TypeInformation.of(new TypeHint
() { }), TypeInformation.of(new TypeHint
() { }) ); //5、将商家的活动规则进行广播 BroadcastStream
broadcastStream = kafkaSource02 .map(t -> t.split(" ")) .map(t -> new Rule(t[0], Integer.parseInt(t[1]))) .broadcast(mapState); //6、数据处理 useStream.connect(broadcastStream) .process(new KeyedBroadcastProcessFunction
() { private transient MapState
buyPathState; //用于存储用户的流量记录 @Override public void open(Configuration parameters) throws Exception { super.open(parameters); super.open(parameters); MapStateDescriptor
descriptor = new MapStateDescriptor<>( "buy_path" , TypeInformation.of(new TypeHint
() {}), TypeInformation.of(new TypeHint
() {}) ); buyPathState = getRuntimeContext().getMapState(descriptor); } @Override public void processElement(UserAction value, ReadOnlyContext ctx, Collector
out) throws Exception { Integer integer = buyPathState.get(value.getChannel()); //查看当前channel的路径长度是多少 int count; if(null == integer){ //防止空指针异常 count = 1; }else{ count = integer + 1; } buyPathState.put(value.getChannel(),count); //修改state当前channel的路径长度 ReadOnlyBroadcastState
broadcastState = ctx.getBroadcastState(mapState); //获取广播中的数据 Integer path = broadcastState.get(value.getChannel()); Integer noHuodong = 0; if(null == path){ noHuodong = 10000; }else{ noHuodong = path; } if(buyPathState.get(value.getChannel()) > noHuodong){ //数据发送至下游 并从state中删除 out.collect(value.toString()); buyPathState.remove(value.getChannel()); } } @Override public void processBroadcastElement(Rule value, Context ctx, Collector
out) throws Exception { //更新状态 BroadcastState
state = ctx.getBroadcastState(mapState); state.put(value.getChannel(),value.getThreshold()); Iterable
> entries = state.entries(); for (Map.Entry
entry : entries) { System.out.println(entry.getKey()+"\t"+entry.getValue()); } } }).printToErr(); env.execute("metricsCounter"); }}

相信通过两个案例的分享,大家已经对broadcast的使用已经有了基本的掌握,接下来,博主对BroadcastProcessFunction和KeyedBroadcastProcessFunction做下分析对比。

public abstract class BroadcastProcessFunction
extends BaseBroadcastProcessFunction { public abstract void processElement(IN1 value, ReadOnlyContext ctx, Collector
out) throws Exception; public abstract void processBroadcastElement(IN2 value, Context ctx, Collector
out) throws Exception;}
public abstract class KeyedBroadcastProcessFunction
{ public abstract void processElement(IN1 value, ReadOnlyContext ctx, Collector
out) throws Exception; public abstract void processBroadcastElement(IN2 value, Context ctx, Collector
out) throws Exception; public void onTimer(long timestamp, OnTimerContext ctx, Collector
out) throws Exception;}

相同处:首先要注意的是,这两个功能都需要实现processBroadcastElement()用于处理广播侧数据和processElement()用于处理非广播侧元素的方法,通过ctx可以获得以下内容:

(1)允许访问广播状态: ctx.getBroadcastState(MapStateDescriptor<K, V> stateDescriptor)
(2)允许查询元素的时间戳:ctx.timestamp(),
(3)得到当前的水印: ctx.currentWatermark()
(4)获得当前处理时间:ctx.currentProcessingTime(),和
(5)将元素发射到侧面输出:ctx.output(OutputTag outputTag, X value)。
同时需要注意的是:stateDescriptor在getBroadcastState()应该与.broadcast(stateDescriptor) 是同一个。

不同处:不同之处在于每个人对广播状态的访问类型。广播方(processBroadcastElement)对此具有读写访问权限,而非广播方(processElement)具有只读访问权限。原因是在Flink中没有跨任务通信。因此,为确保在我们的运行的所有并行实例中的广播状态内容相同,我们仅向广播端提供读写访问权限,广播端在所有任务中看到的元素相同,因此我们需要对每个任务进行计算该端的传入元素在所有任务中都相同。忽略此规则将破坏状态的一致性保证,从而导致结果不一致,并且常常难以调试结果。

最后,由于的事实KeyedBroadcastProcessFunction是,它在键控流上运行,因此它公开了某些功能,这些功能不适用于BroadcastProcessFunction。如下:

(1)所述ReadOnlyContext的processElement()方法可以访问flink的底层定时器服务,其允许注册事件和/或处理时间的定时器。当计时器触发时,onTimer()会使用调用, OnTimerContext该公开了与ReadOnlyContextplus 相同的功能。询问触发的计时器是事件还是处理时间的能力,并且查询与计时器关联的键。
(2)所述Context的processBroadcastElement()方法包含方法 applyToKeyedState(StateDescriptor<S, VS> stateDescriptor, KeyedStateFunction<KS, S> function)。这允许一个注册KeyedStateFunction将被施加到所有键的所有状态与所提供的相关联stateDescriptor。

注意: 注意:仅在“ KeyedBroadcastProcessFunction”的“ processElement()”处才可以注册计时器。在processBroadcastElement()方法中是不可能的,因为没有与广播元素关联的键。

加下来分享一下官网中提到的关于该API使用的注意事项:

(1)没有跨任务通信:如前所述,这就是为什么仅a的广播方 (Keyed)-BroadcastProcessFunction可以修改广播状态的内容的原因。此外,用户必须确保所有任务对于每个传入元素都以相同的方式修改广播状态的内容。否则,不同的任务可能具有不同的内容,从而导致结果不一致。

(2)广播状态中事件的顺序在各个任务之间可能有所不同:尽管广播流的元素保证了所有元素(最终)将进入所有下游任务,但元素可能以与每个任务不同的顺序到达。因此,每个传入元素的状态更新必须不取决于传入事件的顺序。

(3)所有任务都会检查其广播状态:尽管发生检查点时,所有任务在其广播状态中具有相同的元素(检查点屏障不会越过元素),但所有任务都将指向其广播状态,而不仅仅是其中一个。这是一项设计决策,要避免在还原过程中从同一文件读取所有任务(从而避免出现热点),尽管这样做的代价是将检查点状态的大小增加了p倍(=并行度)。Flink保证在还原/缩放后不会重复,也不会丢失数据。在使用相同或更小的并行度进行恢复的情况下,每个任务都会读取其检查点状态。扩展后,每个任务都会读取自己的状态,其余任务(p_new-p_old)以循环方式读取先前任务的检查点。

(4)没有RocksDB状态后端:在运行时将广播状态保留在内存中,并且应该相应地进行内存配置。这适用于所有操作员状态。

转载地址:http://iimzi.baihongyu.com/

你可能感兴趣的文章
队列(数据结构)
查看>>
Mule ESB-Basic Studio Tutorial
查看>>
Mule ESB-Content-Based Routing Tutorial(1)
查看>>
Mule ESB-Content-Based Routing Tutorial(2)
查看>>
Mule ESB-Content-Based Routing Tutorial(3)
查看>>
年末项目经验总结
查看>>
做事情要放下面子,拿起责任
查看>>
敏捷开发实践(1)-故事工作量估算导致的问题
查看>>
记一次解决jenkins持续构建,自动部署的问题
查看>>
敏捷开发实践(2)-要不要文档?
查看>>
敏捷开发实践(3)-我们为什么需要持续集成?
查看>>
《java系统性能调优》--2.缓存
查看>>
JAVA注解引发的思考
查看>>
//基于redisTemplate工具类RedisHelper
查看>>
SpringBoot+RabbitMQ
查看>>
写博意味着什么
查看>>
一些时间管理法则和我自己的时间管理法则
查看>>
变量的声明和作用域——VB
查看>>
VB总结
查看>>
静态数组的声明与例子练习
查看>>