背景介绍
上一章和带大家了解了一下Iceberg的元数据文件特殊之处,也简单的给大家描述了一下Iceberg是如何从快速定位到数据文件的,上一章将的比较干,因为都是一些理论知识,这一章我们从iceberg-flink
模块的源码出发,带大家更加深入的了解Iceberg
注意:本次源码分析基于Iceberg 0.11x分支,主要是讲解iceberg-flink
模块,其余模块因为暂未深入了解所以会跳过,敬请见谅;并且如果有任何地方讲述不当,请直接指出
另外,需要对Iceberg和Flink都一定的基础,否则会出现一知半解的情况
源码分析
开始之前
先回顾一下,我们如何通过Flink去读取一张Iceberg表
首先我们需要建一个类型是iceberg
的Flink Catalog
CREATE CATALOG iceberg_catalog
WITH (
'type'='iceberg',
'catalog-type'='hive',
'uri'='thrift://localhost:9083',
'clients'='5',
'property-version'='1',
'warehouse'='hdfs://hacluster/user/hive/warehouse'
);
假设,我们已经有一张Iceberg的表,通过Hive语句desc formatted iceberg_db.iceberg_kafka_test
得到的信息是这样的
接下来,我们通过执行一条Flink Sql来进行数据的读取
select * from iceberg_catalog.iceberg_db.iceberg_kafka_test
/*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-snapshot-id'='3821550127947089987')
然后耐心等待,我们很快就可以看到源源不断的数据在控制台进行展示。
简单两步,我们就可以通过Flink Sql对一张Iceberg进行数据读取,那么在这背后,都发生了什么事情呢?
CREATE ICEBERG CATALOG
之前在讲Flink 自定义Redis Lookop Table Source的时候说过,Flink会通过JAVA的SPI机制,将所有org.apache.flink.table.factories.TableFactory
的实现类加载,然后通过type
来区分到底是哪个实现类
同样,CREATE ICEBERG CATALOG
也不例外,根据以上步骤,定位到类org.apache.iceberg.flink.FlinkCatalogFactory
然后根据方法createCatalog()
创建我们的Catalog Class org.apache.iceberg.flink.FlinkCatalog
,并且因为我们创建的是Iceberg中的Hive Catalog,所以将Hive CatalogLoader传入FlinkCatalog对象中。
protected Catalog createCatalog(String name, Map<String, String> properties, Configuration hadoopConf) {
CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf);
String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, "default");
String[] baseNamespace = properties.containsKey(BASE_NAMESPACE) ?
Splitter.on('.').splitToList(properties.get(BASE_NAMESPACE)).toArray(new String[0]) :
new String[0];
boolean cacheEnabled = Boolean.parseBoolean(properties.getOrDefault(CACHE_ENABLED, "true"));
return new FlinkCatalog(name, defaultDatabase, baseNamespace, catalogLoader, cacheEnabled);
}
接下来,我们继续来看FlinkCatalog这个类。
在实例化它的时候,会调用传入的Hive CatalogLoader对象,并执行它loadCatalog()
的方法,得到真正的HiveCatalog
,它的作用是负责与Hive Metastore通信,去执行我们的建表、删表、查表语句,这里就不细讲了,具体可以看看iceberg-hive-metastore
模块
回到FlinkCatalog,它还提供了listDatabases
、listTables
等方法,对应着我们的Sql语句show databases;
、show tables
,无一例外都是通过上面说的真正的HiveCatalog
去执行对应的操作,有兴趣的同学可以去看看,就不在这里展开说了
FlinkCatalog 中有个特殊的方法getTableFactory()
,它的作用是将FlinkTableFactory的实例对象返回出去。那么,这个Factory是啥呢?点进去看看,发现了两个眼熟的方法createTableSource()
、createTableSink()
。
那既然本篇的主题是读过程分析,那么我们就对createTableSource()
中的内容进行深入分析
CREATE TABLE SOURCE
createTableSource()
方法很简单,一共4行代码
@Override
public TableSource<RowData> createTableSource(TableSourceFactory.Context context) {
ObjectPath objectPath = context.getObjectIdentifier().toObjectPath();
TableLoader tableLoader = createTableLoader(objectPath);
TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(context.getTable().getSchema());
return new IcebergTableSource(tableLoader, tableSchema, context.getTable().getOptions(),
context.getConfiguration());
}
其中,前三行的作用分别是拿到表路径(一般来说就是库名.表名),表加载对象(负责加载Iceberg的表)、表结构,然后通过它们去实例化IcebergTableSource,所以接下来我们再来看看IcebergTableSource这个类
点进去之后,我们能看到它实现了StreamTableSource这个接口,也就是说,IcebergTableSource提供对其数据的读取访问的能力,也就是通过getDataStream()
去返回一个dataStream,供下游的operator消费
@Override
public DataStream<RowData> getDataStream(StreamExecutionEnvironment execEnv) {
return FlinkSource.forRowData()
.env(execEnv)
.tableLoader(loader)
.properties(properties)
.project(getProjectedSchema())
.limit(limit)
.filters(filters)
.flinkConf(readableConfig)
.build();
}
这些链式方法的作用基本上都是把值塞进去,我们重点看一下properties()
和build()
properties()
里面的内容很简单
public Builder properties(Map<String, String> properties) {
contextBuilder.fromProperties(properties);
return this;
}
我们来看看fromProperties()
中做了什么
Builder fromProperties(Map<String, String> properties) {
Configuration config = new Configuration();
properties.forEach(config::setString);
return this.useSnapshotId(config.get(SNAPSHOT_ID))
.caseSensitive(config.get(CASE_SENSITIVE))
.asOfTimestamp(config.get(AS_OF_TIMESTAMP))
.startSnapshotId(config.get(START_SNAPSHOT_ID))
.endSnapshotId(config.get(END_SNAPSHOT_ID))
.splitSize(config.get(SPLIT_SIZE))
.splitLookback(config.get(SPLIT_LOOKBACK))
.splitOpenFileCost(config.get(SPLIT_FILE_OPEN_COST))
.streaming(config.get(STREAMING))
.monitorInterval(config.get(MONITOR_INTERVAL))
.nameMapping(properties.get(DEFAULT_NAME_MAPPING));
}
很明显能看出来,我们通过传入的properties
的值,来指定我们的参数,例如startSnapshotId
、asOfTimestamp
等等;那properties
的值都是什么呢?
一部分值来自于我们建表的时候,WITH
中指定的参数;一部分是我们通过Table Hints动态修改的参数,比如start-snapshot-id
、monitor_interval
说完properties()
,我们来看build()
,这里面的内容很多,大家认真看
public DataStream<RowData> build() {
Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null");
FlinkInputFormat format = buildFormat();
ScanContext context = contextBuilder.build();
TypeInformation<RowData> typeInfo = RowDataTypeInfo.of(FlinkSchemaUtil.convert(context.project()));
if (!context.isStreaming()) {
int parallelism = inferParallelism(format, context);
return env.createInput(format, typeInfo).setParallelism(parallelism);
} else {
StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context);
String monitorFunctionName = String.format("Iceberg table (%s) monitor", table);
String readerOperatorName = String.format("Iceberg table (%s) reader", table);
return env.addSource(function, monitorFunctionName)
.transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format));
}
}
该方法主要做了两件事情
-
利用
tableLoader
加载对应的table
,然后通过这个table
获取到对应的FileIO、Schema、EncryptionManager;再加上之前的fromProperties()
方法构建出的ScanContext对象,一起组装成了负责辅助DataSource读取数据、分发数据的InputFormat -
将DataSource
StreamingMonitorFunction
注册到env
上,并且接上了一个自定义算子StreamingReaderOperator
至此,CREATE TABLE SOURCE的逻辑已经说完,接下来我们再看看Flink如何去mointor&read Iceberg表的
Mointor & Read
先看StreamingMonitorFunction
可以看到它实现了CheckpointedFunction接口,所以能够保证在source端的一致性;
另外,因为它并没有实现ParallelSourceFunction接口,所以它注定只能有一个并行度。这里的目的是确保在只有一个线程去监控Iceberg表和分发任务,多线程只会发生数据错乱
每处理完当前到表最新的快照id中的数据之后,将表最新快照id标记为最后一次处理的快照id,并在checkpoint的时候,存在到state中;如果程序从故障中恢复,则取state中的快照id作为起始的快照id
这里有一个让我产生疑惑的地方
if (context.isRestored()) {
LOG.info("Restoring state for the {}.", getClass().getSimpleName());
lastSnapshotId = lastSnapshotIdState.get().iterator().next();
}
如果lastSnapshotIdState中没有任何值,lastSnapshotIdState.get().iterator().next()是否会抛出异常?
接下来看一下monitorAndForwardSplits()
方法,它是负责去监控和分发任务
FlinkInputSplit[] splits = FlinkSplitGenerator.createInputSplits(table, newScanContext);
for (FlinkInputSplit split : splits) {
sourceContext.collect(split);
}
核心代码就这2句话,首先通过FlinkSplitGenerator的createInputSplits
方法,利用传入的表,和新构建的ScanContext去构建任务分片,再将每个任务分发到下游算子StreamingReaderOperator中
FlinkSplitGenerator中的createInputSplits
就不细看了,绝大部分内容(99.99%)属于Iceberg的核心逻辑,大体流程也和我们上次分析Iceberg文件中所涉及的读逻辑差不多太多。这边我还没有细看,之后讲到Iceberg核心技术的时候再来分析。
接下来再看看StreamingReaderOperator是如何消费的
和StreamingMonitorFunction一样,StreamingReaderOperator也实现了checkpoint相关的接口,因此也有一致性的保障;不一样的地方是这个算子可以指定任意并行度
在内存中存放的任务分片splits是个队列,每次在接收新的task就会丢到里面,每次会从splits中取出一个任务进行处理,再处理完一个任务后,进行类似递归的操作,将splits中的任务不断取出,在读完splits中的任务会将算子标记为IDLE状态,
在处理每个task的时候,之前的format会用task去构建一个RowDataIterator对象,并且根据task的文件格式去创建不同格式的Iterable对象,并且会根据是否有位置删除和等值删除进行数据过滤,保证数据的一致性
然后将Iterable对象中的每个元素不断取出,发向下游
这块儿的逻辑不算太复杂,就是比较碎,有兴趣的可以自己去翻看,大概逻辑就是我讲的那样
好了,讲到这里,Iceberg中的Flink读过程基本上都看完了,有些没有讲到的地方大家也可以自己去看,另外涉及Iceberg核心逻辑的地方之后的分享也会将
写在最后
- 本章的内容还是比较多的,大家要慢慢看,最好是结合源码进行debug才能更好的了解Iceberg中的Flink读过程
- 最近比较忙,所以这周就分享这么一篇了。另外我自己实现了如何给Flink Sql中的每个算子指定并行度、chain策略以及UID,有机会会分享出来,特别是指定UID,对我们的意义很大,这样我们在Flink Sql进行逻辑变更的时候,也能从checkpoint/savepoint中恢复数据,以前因为UID是随机生成的无法自己指定,只要逻辑变更,哪怕是同一个算子,产生的UID也会不一样,所以导致无法恢复。
- 下一章我会暂停对Iceberg的分享,转而去分享一下Hudi的相关内容,有喜欢Hudi的同学不要错过
- 最后,看在我写这么多的分子上,点个赞呗?