Paimon读取流程
查询模式
先来看看官网关于Paimon查询模式的说明
可以看到查询模式围绕snapshot展开, 而snapshot分了两种一种是Last compact snapshot和 last snapshot. 直接读last snapshot的话应该是需要merge on read. 而last compact snapshot 应该有点类似于hudi里面的Read Optimized Queries
Change log producer
上表中流读都提到需要读取变更流的数据.因此,Paimon表要有产生变更流的数据的能力. 内置了几种不同的changelog producer.
Changelog producer的含义是这张Paimon表的change log producer. 也就是用户写入数据后,如果对于这张表产生正确的change log. 这样下游才可以基于这个变更流进行增量的处理.
None
不单独产生Changelog文件. 按照官网的说法是只能看到snapshot之间的变化. 但是没有old value.
Paimon source can only see the merged changes across snapshots, like what keys are removed and what are the new values of some keys.
Input
在刷写数据的时候,会同时写一份Changelog的文件,提供给下游消费. 相当于为了流式能消费到变更流的视图, 需要将上游的变更流数据另外保存一份.
使用这种类型的clp的前提是, 基于输入数据已经能完全反应这张表的Changelog, 例如由CDC同步进来的数据,是可以的. 但是对于partial update 是不行的
// 如果配置了ChangelogProducer.INPUT 那么再刷写WriteBuffer的时候会同时将原始数据写入到changelog里面
final RollingFileWriter<KeyValue, DataFileMeta> changelogWriter =
changelogProducer == ChangelogProducer.INPUT
? writerFactory.createRollingChangelogFileWriter(0)
: null;
final RollingFileWriter<KeyValue, DataFileMeta> dataWriter =
writerFactory.createRollingMergeTreeFileWriter(0);
try {
// forEach会对原始数据的基于key的有序遍历
writeBuffer.forEach(
keyComparator,
mergeFunction,
changelogWriter == null ? null : changelogWriter::write,
dataWriter::write); // 最终使用的Orc/Parquet Writer来将数据写出
} finally {
if (changelogWriter != null) {
changelogWriter.close();
}
dataWriter.close();
}
Lookup
当input无法形成一个完整的changelog, 比如partial update的场景中, 每个单独的input是没法产生changelog流的, changelog的过程实际和Compaction的merge过程相关
// 针对lookup的changelog producer 需要
// 1. 使用LookupCompaction CompactionStrategy
// 2. 使用LookupMergeTreeCompactRewriter
// 3. 使用LookupMergeFunction
LOOKUP(
"lookup",
"Generate changelog files through 'lookup' before committing the data writing.");
KeyValue result = mergeFunction.getResult();
checkArgument(result != null);
KeyValue highLevel = mergeFunction.highLevel;
boolean containLevel0 = mergeFunction.containLevel0;
// 1. No level 0, just return
// 没有level 0的数据, 意味着没有新数据产生
// 那么没有changelog文件产生, 只是高层文件的合并
if (!containLevel0) {
return reusedResult.setResult(result);
}
// 2. With level 0, with the latest high level, return changelog
// 先前的value也在此次的Compaction列表里面,直接就可以得出change log了
if (highLevel != null) {
setChangelog(highLevel, result);
return reusedResult.setResult(result);
}
// 3. Lookup to find the latest high level record
// 向更高level中查找这个key先前的数据, 为了产生变更流代价还是挺高的
// org.apache.paimon.mergetree.LookupLevels#lookup
highLevel = lookup.apply(result.key());
if (highLevel != null) {
mergeFunction2.reset();
mergeFunction2.add(highLevel);
mergeFunction2.add(result);
result = mergeFunction2.getResult();
setChangelog(highLevel, result);
} else {
setChangelog(null, result);
}
return reusedResult.setResult(result);
大致过程就是在Compaction的过程中会向高层的文件中查找该key的数据, 并根据查找结果来构建change log stream.
因为高层文件的key是有序的, 所以会通过二分法来过滤文件meta,快速定位到属于哪个文件. 但是因为这个文件是Parquet/Orc的列存文件, 无法直接根据key去高效查询的. 所以会先将原始数据读出,并重新成一个新的格式的文件,用于lookup探查, 主要是构建key的索引, 用于. HashLookupStoreWriter
HashLookupStoreReader
主体逻辑可以参看
Full-compaction
从上面的过程分析可以看出lookup的clp开销还是很大的,需要重读某个key的数据, 然后重新构建file cache, 再写出. 这里还提供了 full-compaction
的方式.同lookup一样,这个也是在compaction阶段来产生的, 不过是full Compaction阶段.
Full compaction changelog producer can produce complete changelog for any type of source. However it is not as efficient as the input changelog producer and the latency to produce changelog might be high.
他可以支持任意类型的input,但是时延会比较高. 10min 往上
实现类 FullChangelogMergeTreeCompactRewriter
和 FullChangelogMergeFunctionWrapper
full compaction的时候不会产生delete的change log消息(大概是因为并不知道谁被delete了?)
在Full compaction阶段最后数据都会写到top level. 然后将最后合并后的数据和topLevel比较, 然后得出一个变更消息写到change log文件中.
对于离线场景的一般delete消息的需求
昨天新增今天删,昨天日增量分区有,今天增量分区没有 (也就是change log中并没有delete消息).昨天的日全量有,今天的日全量没有
今天新增今天删,今天的日增量分区没有,今天的日全量也没有
批读
批模式 | 流模式 | |
---|---|---|
latest-full | 读取最新的snapshot. 获取的是最近一次的snapshot | 先读取最新的snapshot, 然后持续读取变更流 |
compacted-full | 读取最近一次Compaction之后的snapshot. | |
获取的snapshot是最近一次compaction的. 理论上这样读取阶段就不需要Merge On Read了 | 先读取最近一次Compaction之后的snapshot, 然后持续读取变更流 | |
latest | 和latest-full一样 | 只读取最新变化的数据, 没有读取snapshot |
from-timestamp | 读取一个早于或等于 scan.timestamp-millis 指定时间戳的snapshot |
读取某个时间之后的数据, 不读取snapshot |
from-snapshot | 读取scan.snapshot-id 指定的某个snapshot id |
读取某个snapshot之后的数据, 不读取snapshot |
from-snapshot-full | 读取scan.snapshot-id 指定的某个snapshot id |
先读取某个snapshot, 然后持续读取其后的变化数据 |
PK表
StaticFileStoreSource
org.apache.paimon.table.source.AbstractInnerTableScan#createStartingScanner
InnerTable#newScan#plan (返回的Splits列表)
org.apache.paimon.table.AbstractFileStoreTable#newScan
org.apache.paimon.KeyValueFileStore#newScan()
org.apache.paimon.table.source.snapshot.SnapshotSplitReaderImpl#splits
org.apache.paimon.operation.AbstractFileStoreScan#plan 通过snapshot, 读取到相应的ManifestEntry 过滤出所有要读的文件
org.apache.paimon.table.source.snapshot.SnapshotSplitReaderImpl#generateSplits 对文件列表创建splits
org.apache.paimon.table.source.MergeTreeSplitGenerator#split 每个bucket内部进行splits切分, 提高读取的并行度
org.apache.paimon.flink.source.FileStoreSourceSplitReader
org.apache.paimon.table.source.KeyValueTableRead#createReader
org.apache.paimon.operation.KeyValueFileStoreRead#createReaderWithoutOuterProjection Merge On Read
Append 表
大体上和上面一样. 除了切分split的时候和创建reader的时候
org.apache.paimon.table.source.AppendOnlySplitGenerator#split
org.apache.paimon.operation.AppendOnlyFileStoreRead#createReader
流读
org.apache.paimon.table.AbstractFileStoreTable#newStreamScan
org.apache.paimon.table.source.AbstractInnerTableScan#createStartingScanner 创建一个初始的scan 这个和批模式很类似. 但是大部分流读都不会去读取Snapshot, 这个部分只是生成一个next Snapshot的id
org.apache.paimon.table.source.InnerStreamTableScanImpl#createFollowUpScanner 创建一个变更流的scan
变更流就和上面的Changelog producer息息相关, 每一种clp都有一个对应的变更流的planner. 用于根据Snapshot返回splits
- DeltaFollowUpScanner
- InputChangelogFollowUpScanner
- CompactionChangelogFollowUpScanner
- CompactionChangelogFollowUpScanner
并且也可以看到变更流的消费是跟着Snapshot走的, 在Stream 的 Source中会定期去获取splits, 就会触发定期Plan的获取, Plan的获取依赖于Snapshot. 所以读取的时延实际上Snapshot息息相关, 而Snapshot的产生又和上游的Checkpoint频率息息相关.
对于Append表 changelog 应该是delta 的数据, 是不是Append表应该只有DeltaFollowUpScanner 呢?
Lookup Join
Paimon还支持维表关联. 维表关联只支持all的模式. 会将数据全部load到本地(会有一些过滤下推), 并存储到Rocksdb中. 不会在关联的过程中直接去查询文件, 从上面的lookup changelog producer实现中也可以看出 kv的查询开销还是很大的.
参考
changelog-producer: https://paimon.apache.org/docs/master/concepts/primary-key-table/#changelog-producers
Paimon读取流程的更多相关文章
- HDFS写入和读取流程
HDFS写入和读取流程 一.HDFS HDFS全称是Hadoop Distributed System.HDFS是为以流的方式存取大文件而设计的.适用于几百MB,GB以及TB,并写一次读多次的场合.而 ...
- 【Spring源码分析】配置文件读取流程
前言 Spring配置文件读取流程本来是和http://www.cnblogs.com/xrq730/p/6285358.html一文放在一起的,这两天在看Spring自定义标签的时候,感觉对Spri ...
- Android系统分析之运营商显示流程分析之运营商信息的读取流程二
运营商显示流程分析之运营商信息的读取流程 一. SIM卡运营商信息的读取 从前面的 运营商信息的获取和赋值 可以知道SIM卡运营商的赋值最终是在 SIMRecords 中完成的, 而SIM卡信息的相关 ...
- 8.hbase写入流程和读取流程
1 hbase写入流程 hbase中无论是新增数据还是修改已有行,其内部流程都是一样的,hbase执行写入时会写到两个地方,write-ahead log 简称wal 也叫hlog 预写式日志 和 M ...
- (第二章第一部分)TensorFlow框架之文件读取流程
本章概述:在第一章的系列文章中介绍了tf框架的基本用法,从本章开始,介绍与tf框架相关的数据读取和写入的方法,并会在最后,用基础的神经网络,实现经典的Mnist手写数字识别. 有四种获取数据到Tens ...
- HBase读取与写入流程
写入流程 读取流程 https://yq.aliyun.com/articles/670748?spm=a2c4e.11153940.blogcont684011.28.427e4648CTtaPL
- 2017.2.28 activiti实战--第五章--用户与组及部署管理(三)部署流程及资源读取
学习资料:<Activiti实战> 第五章 用户与组及部署管理(三)部署流程及资源读取 内容概览:如何利用API读取已经部署的资源,比如读取流程定义的XML文件,或流程对应的图片文件. 以 ...
- [spring源码学习]三、IOC源码——自定义配置文件读取
一.环境准备 在文件读取的时候,第9步我们发现spring会根据标签的namespace来选择读取方式,联想spring里提供的各种标签,比如<aop:xxx>等应该会有不同的读取和解析方 ...
- activiti自定义流程之整合(五):启动流程时获取自定义表单
流程定义部署之后,自然就是流程定义列表了,但和前一节一样的是,这里也是和之前单独的activiti没什么区别,因此也不多说.我们先看看列表页面以及对应的代码,然后在一步步说明点击启动按钮时如何调用自定 ...
- shell启动时读取的配置文件
bash shell具体可以分为3种类型,这3种类型为: 1 login shell 就是需要输入用户名和密码才能登陆的shell 2 可交互的非login shell 就是不用登陆的,但是可以同用户 ...
随机推荐
- Android Banner - ViewPager 02
Android Banner - ViewPager 02 现在来给viewpager实现的banenr加上自动轮播 自动轮播的原理,使用handler的延迟消息来实现. 自动轮播实现如下内容 开始轮 ...
- Go For Web:踏入Web大门的第一步——Web 的工作方式
前言: 本文作为解决如何通过 Golang 来编写 Web 应用这个问题的前瞻,对 Golang 中的 Web 基础部分进行一个简单的介绍.目前 Go 拥有成熟的 Http 处理包,所以我们去编写一个 ...
- 用Abp实现找回密码和密码强制过期策略
@ 目录 重置密码 找回密码 发送验证码 校验验证码 发送重置密码链接 创建接口 密码强制过期策略 改写接口 Vue网页端开发 重置密码页面 忘记密码控件 密码过期提示 项目地址 用户找回密码,确切地 ...
- GPT-4:思考的曙光还是数据的缩影?
海盗分金,GPT-4初露锋芒 GPT系列模型横空出世后,其是否真实具有思考和推理的能力一直被业界关注.GPT-3.5在多条狗问题和海盗分金问题上表现糟糕.GPT-4在这两个谜题上给出的答案令人惊喜,甚 ...
- 1778D Flexible String Revisit
1778D Flexible String Revisit 目录 1778D Flexible String Revisit 题目大意: 做法: dp 注意 code 题目大意: 给你两个长度均为\( ...
- 轻量化3D文件格式转换HOOPS Exchange新特性
BIM与AEC市场发展现状 近年来BIM(建筑信息模型)和AEC(建筑.工程和施工)市场一直保持着持续增长.2014 年全球 BIM 软件市场价值 27.6 亿美元,而到 2022年,预期到达115. ...
- Prism Sample 7 Modules LoadManual
这种模块是手动载入的,需要的时候手动加载. 在app.xaml.cs中注册为按需加载,代码 protected override void ConfigureModuleCatalog(IModule ...
- 2022-04-19:A*算法, 过程和Dijskra高度相处, 有到终点的预估函数, 只要预估值<=客观上最优距离,就是对的。 预估函数是一种吸引力: 1)合适的吸引力可以提升算法的速度; 2)吸引
2022-04-19:A*算法, 过程和Dijskra高度相处, 有到终点的预估函数, 只要预估值<=客观上最优距离,就是对的. 预估函数是一种吸引力: 1)合适的吸引力可以提升算法的速度; 2 ...
- Element Cascader 级联选择器去除空叶子节点
此处以后端获取部门级联List为例 以下为数据结构 { data: { children: [ 0:{childre:[ 0:{}, 1:{} ]}, 1:{}, 2:{}, 3:{}, 4:{}, ...
- 【汇编】masm文件夹整理
整理masm文件夹 前言 不好意思,我又来了,今天上汇编课,发现汇编masm文件夹实在是太乱了,像这样: 着实太乱了!程序一多就会很乱很让人心烦!(虽然我现在没有认真上汇编课吧,就这几个文件) 开始折 ...