起因
最近临近双十一,你们也知道,电商类公司到双十一的时候有多忙。压测、稳定性、实时大屏,一堆事情要在双十一之前完成。加上我们最近在做数据平台相关的事情,简直忙到爆炸。就在这么忙的时候,还踩到了Flink中Topn的坑。issule我已经提了,具体可以看这个点我直达issule,感谢@云邪老师帮我改描述标题和内容,我英文实在蹩脚
简单的说一下场景
Data Source : kafka + canal-json
Data Process : Topn
转换成Sql就是
create table t_order(
`id` int,
`updated_at` string,
`amount` decimal(38,14)
) WITH (
'connector' = 'kafka',
'topic' = 'test',
'properties.bootstrap.servers' = '127.0.0.1:9092',
'format' = 'canal-json',
'canal-json.ignore-parse-errors' = 'true',
'scan.startup.mode' = 'latest-offset'
);
select sum(amount) from (
select *,
row_number() over(partition by id order by updated_at desc) rn
from t_order a
) t
where t.rn = 1
group by id
模拟
先启动我们的代码,然后在Kafka 命令行执行
# t.log里面的内容就是demo数据
./kafka-console-producer.sh --broker-list 127.0.0.1:9092 --topic test < t.log
Demo数据
{"data":[{"id":"1","amount":"197.60","updated_at":"2020-11-11 11:11:11"}],"old":null,"type":"INSERT"}
{"data":[{"id":"1","amount":"197.60","updated_at":"2020-11-11 11:11:12"}],"old":[{"updated_at":"2020-11-11 11:11:11"}],"type":"UPDATE"}
{"data":[{"id":"1","amount":"197.60","updated_at":"2020-11-11 11:11:12"}],"old":[{"updated_at":"2020-11-11 11:11:11"}],"type":"UPDATE"}
{"data":[{"id":"1","amount":"197.60","updated_at":"2020-11-11 11:11:13"}],"old":[{"updated_at":"2020-11-11 11:11:12"}],"type":"UPDATE"}
其中第二条json和第三条是一模一样的两条
观察我们的控制台,看看输出
// 我只把所有的print数据和关键StackTrace贴了出来
aaaa> (true,197.60000000000000)
aaaa> (false,197.60000000000000)
aaaa> (true,197.60000000000000)
…
Caused by: java.lang.RuntimeException: Can not retract a non-existent record. This should never happen.
at org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.processElement(RetractableTopNFunction.java:184)
at org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.processElement(RetractableTopNFunction.java:57)
at
…
很明显,处理到第三条canal-json时报错了,接下来让我们走进源码,开始排查问题
排查
看到报错,我们完全不慌,打开源码定位到org.apache.flink.table.runtime.operators.rank.RetractableTopNFunction.RetractableTopNFunction.processElement(…)
简单的给大家说一下这个方法干啥的
// a map state stores mapping from sort key to records list
private transient MapState<RowData, List<RowData>> dataState;
// a sorted map stores mapping from sort key to records count
private transient ValueState<SortedMap<RowData, Long>> treeMap;
// 这2个state,一个负责记录 同一个sort key来过的所有记录,一个记录这个key来过多少次
知道了这2个之后,再接着看
boolean isAccumulate = RowDataUtil.isAccumulateMsg(input);
public static boolean isAccumulateMsg(RowData row) {
RowKind kind = row.getRowKind();
return kind == RowKind.INSERT || kind == RowKind.UPDATE_AFTER;
}
input是输入的数据,也就是将每条canal-json反序列化后的数据,对于UPDATE
类型的数据,会反序列化成两条数据
代码会根据input数据判断这条是INSERT、UPDATE_BEFORE、UPDATE_AFTER、DELETE
中的哪个。
如果是INSERT、UPDATE_AFTER
则 treeMap中,当前key(根据input数据推算出) 对应的treeMap +1,当前的input插入dataState的list中,并下发当前input;
如果是UPDATE_BEFORE、DELETE
则撤回当前input,接下来关键的地方来了
如果sortedMap包含当前key,那么没问题,取出值-1然后判断是否等于0再塞回去
如果不包含,会先判断sortedMap是否为空?为空则说明可能是因为清理state导致,跳过;如果不为空,则抛出异常
Can not retract a non-existent record. This should never happen.
为啥要直接抛出异常呢?
我猜想是因为,代码的原作者觉得,不可能出现UPDATE_BEFORE、DELETE
要删除的数据没有出现过;也就是说,DELETE
或者UPDATE_BEFORE
不存在的数据,这件事本身就是不该发生的
但这是有可能的,在我们用canal或者maxwell这种binlog同步工具,他们并不能做到exactly-once,所以有重复数据很正常;当然,我们也可以在做topN之前先用别的方式去重,具体可以看我这个博客精准去重。但是这样就多了一层处理,比较麻烦,而且TopN本身就是用来做去重的,这样做不是多次一举吗?
别慌,山人自有妙计,我们接着往下看
解决
找到了问题,接下来我们来看看如何解决
思路很简单,把重复的数据跳过就ok了。关键在于如何判断数据是重复的?
想一想,UPDATE_BEFORE
后面必然是跟着UPDATE_AFTER
的,也就是说,这2条数据是肯定会按着顺序来的。只要我们找到要跳过的UPDATE_BEFORE
,然后标记一下,下一条来的数据要跳过。当处理下一条数据时,执行跳过就行。
What are u talking about?
让我们代码见真章
private ValueState<Long> state;
public void open(Configuration parameters) throws Exception {
super.open(parameters);
……
MapStateDescriptor<RowData, Long> timeMapStateDescriptor = new MapStateDescriptor<>("time-state", RowData.class, Long.class);
timeState = getRuntimeContext().getMapState(timeMapStateDescriptor);
}
public void processElement(RowData input, Context ctx, Collector<RowData> out) throws Exception {
long currentTime = ctx.timerService().currentProcessingTime();
……
Long value = state.value();
System.out.println("value :" + value);
if (value != null && sourceRowkind.equals("+U")) {
state.update(null);
return;
}
state.update(null);
// register state-cleanup timer
registerProcessingCleanupTimer(ctx, currentTime);
initRankEnd(input);
……
if (sortedMap.isEmpty()) {
if (lenient) {
LOG.warn(STATE_CLEARED_WARN_MSG);
} else {
throw new RuntimeException(STATE_CLEARED_WARN_MSG);
}
} else {
// throw new RuntimeException(
// "Can not retract a non-existent record. This should never happen.");
LOG.warn("Can not retract a non-existent record. This should never happen. Maybe has duplicate data");
if ("-U".equals(sourceRowkind)) {
state.update(1L);
}
}
}
}
treeMap.update(sortedMap);
}
不关键的代码我都删除了,具体思路就是我说的那样:我用了一个mapState,来标记当前key对面的下一条数据要不要跳过
改完之后让我们编译一下源码并重新跑一下刚才的任务
cd ${FLINK_PROJECT_HOME}/flink-table/flink-table-runtime-blink
mvn clean install -Drat.skip=true -DskipTests -Dcheckstyle.skip=true
重新执行任务,并发送我们的demo数据,看一下结果
aaaa> (true,197.60000000000000)
aaaa> (false,197.60000000000000)
aaaa> (true,197.60000000000000)
aaaa> (false,197.60000000000000)
aaaa> (true,197.60000000000000)
确实把我们的一条canal-json数据跳过了,看样子没问题
不过真的没问题吗?我给大家留了个坑,如果只是这样解决,在当前的sql下确实没问题,但是如果把聚合去掉,直接select *
,那么还是会有别的问题,怎么解决的,我就先不说了,下回再给大家揭秘(其实是不想写了)
打广告
写在最后,打个广告吧
大家在杭州的,有换工作打算的,可以钉钉联系我
公司叫 浙江单创品牌管理有限公司 ,具体公司信息可以Boss、脉脉看,需要内推的话加我,毕竟内推几率大一点。
我们组目前急缺前端、大数据测试、数据平台开发、数仓开发、数据应用开发,什么级别都要。别的岗位也可以帮忙内推,有兴趣死命钉我吧