1.基础
IcebergTableSource支持投影下推、filter下推、limit下推,实现了ScanTableSource接口,所以Iceberg实现了这四个接口的对应方法,具备了对应的功能
public class IcebergTableSource
implements ScanTableSource,
SupportsProjectionPushDown,
SupportsFilterPushDown,
SupportsLimitPushDown {
2.SupportsLimitPushDown
Limit下推,接口就一个applyLimit,Iceberg中的实现为
public void applyLimit(long newLimit) {
this.limit = newLimit;
}
这个最终在DataStream读取数据的时候使用
3.SupportsFilterPushDown
public Result applyFilters(List<ResolvedExpression> flinkFilters) {
List<ResolvedExpression> acceptedFilters = Lists.newArrayList();
List<Expression> expressions = Lists.newArrayList();
for (ResolvedExpression resolvedExpression : flinkFilters) {
Optional<Expression> icebergExpression = FlinkFilters.convert(resolvedExpression);
if (icebergExpression.isPresent()) {
expressions.add(icebergExpression.get());
acceptedFilters.add(resolvedExpression);
}
}
this.filters = expressions;
return Result.of(acceptedFilters, flinkFilters);
}
核心跟limit类似,主要是生成了filters,最终也是在DataStream读取数据的时候使用
Result这个是Flink里的结果,应该是在优化器里用的
4.SupportsProjectionPushDown
投影下推,Flink的优化器上是支持嵌套列的下推的,但是需要下游Source实现,目前Iceberg不支持嵌套列
主要两个接口:supportsNestedProjection、applyProjection。supportsNestedProjection就是返回是否支持嵌套列
applyProjection是设置列信息,Flink这里传入的是一个嵌套列的形式,Iceberg这里会把嵌套列简化为非嵌套列
public void applyProjection(int[][] projectFields) {
this.projectedFields = new int[projectFields.length];
for (int i = 0; i < projectFields.length; i++) {
Preconditions.checkArgument(
projectFields[i].length == 1, "Don't support nested projection in iceberg source now.");
this.projectedFields[i] = projectFields[i][0];
}
}
对于嵌套列,如下Table列信息
CREATE TABLE NestedTable (
| id BIGINT,
| deepNested ROW<
| nested1 ROW<name STRING, `value.` INT>,
| `nested2.` ROW<num INT, flag BOOLEAN>>,
| nested ROW<name STRING, `value` INT>,
| name STRING,
| nestedItem ROW<deepArray ROW<`value` INT> ARRAY, deepMap MAP<STRING, INT>>,
| lower_name AS LOWER(name)
|)
如下查询语句
SELECT id,
| deepNested.nested1.name AS nestedName,
| nested.`value` AS nestedValue,
| deepNested.`nested2.`.flag AS nestedFlag,
| deepNested.`nested2.`.num + deepNested.nested1.`value.` AS nestedNum,
| lower_name
|FROM NestedTable
相应的projectFields二维数组

最终产生的projectedFields也是在DataStream读取数据的时候使用,传入之前会基于Catalog的schema做一层(过滤?)
此处应该是做映射,根据select需要获取的列信息,即projectedFields ,获取对应的Iceberg的schema中对应的信息
private TableSchema getProjectedSchema() {
if (projectedFields == null) {
return schema;
} else {
String[] fullNames = schema.getFieldNames();
DataType[] fullTypes = schema.getFieldDataTypes();
return TableSchema.builder()
.fields(
Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new),
Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new))
.build();
}
}
5.ScanTableSource
核心是getScanRuntimeProvider的接口,在这里会创建DataStream,DataStream的实现有两个接口,一个是旧的接口,一个是FLIP27对应的新接口,由配置决定:table.exec.iceberg.use-flip27-source
public DataStream<RowData> produceDataStream(
ProviderContext providerContext, StreamExecutionEnvironment execEnv) {
if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) {
return createFLIP27Stream(execEnv);
} else {
return createDataStream(execEnv);
}
}
FLIP27的实现接口如下,就是设定了大量的参数
private DataStreamSource<RowData> createFLIP27Stream(StreamExecutionEnvironment env) {
SplitAssignerType assignerType =
readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE);
IcebergSource<RowData> source =
IcebergSource.forRowData()
.tableLoader(loader)
.assignerFactory(assignerType.factory())
.properties(properties)
.project(getProjectedSchema())
.limit(limit)
.filters(filters)
.flinkConfig(readableConfig)
.build();
DataStreamSource stream =
env.fromSource(
source,
WatermarkStrategy.noWatermarks(),
source.name(),
TypeInformation.of(RowData.class));
return stream;
}
5.1.getScanRuntimeProvider的使用
这里是Flink的流程,Flink SQL的流程中,最终是调用translateToPlan接口,在这个接口的调用中,最终调用到CommonExecTableSourceScan的translateToPlanInternal
在这里,会获取ScanRuntimeProvider,然后根据类型,创建对应的transform,Iceberg对应的是DataStreamScanProvider。调用produceDataStream就是前面提到的接口,完成了DataStream的创建
} else if (provider instanceof DataStreamScanProvider) {
Transformation<RowData> transformation =
((DataStreamScanProvider) provider)
.produceDataStream(createProviderContext(config), env)
.getTransformation();
meta.fill(transformation);
transformation.setOutputType(outputTypeInfo);
return transformation;
6.IcebergSource
看上节的内容,IcebergSource是Iceberg对Flink数据源的真正实现,在IcebergTableSource当中构建
继承了Flink的Source接口,所以关注两个方法:createReader、createEnumerator
6.1.createReader
Reader最重要的是readerFunction,这个是在build的时候设置的,使用的实现类为RowDataReaderFunction
RowDataReaderFunction rowDataReaderFunction =
new RowDataReaderFunction(
flinkConfig,
table.schema(),
context.project(),
context.nameMapping(),
context.caseSensitive(),
table.io(),
table.encryption());
this.readerFunction = (ReaderFunction<T>) rowDataReaderFunction;
这里有几个核心的参数table.schema()、context.project()、context.nameMapping()
- table.schema()
这是Iceberg的tableLoader加载表后获取的整个表的schema,包含了表的所有列
这个schema还会用于对Flink传入的要查询的列进行id归一化,因为列在传入的id和Iceberg这边的id可能不一样(id就是sql优化器转换过程中常看到的$0这种)
projectedFlinkSchema 这里只有要查询的列
if (projectedFlinkSchema != null) {
contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema));
}
projectedFlinkSchema是前面getProjectedSchema方法获取的结果
- context.project()
这个就是上面contextBuilder.project设置的project
- context.nameMapping()
这个目前看起来没有用,由schema.name-mapping.default配置
这个是在iceberg文件没有field ids信息时,通过配置提供字段映射的
6.2.createEnumerator
6.2.1.SplitAssigner
是核心对象,由table.exec.iceberg.split-assigner-type配置工厂类,默认使用SimpleSplitAssignerFactory,创建的分配器为SimpleSplitAssigner
6.2.2.SplitEnumerator
根据流还是批,会创建不同的对象
if (scanContext.isStreaming()) {
ContinuousSplitPlanner splitPlanner =
new ContinuousSplitPlannerImpl(lazyTable(), scanContext, planningThreadName());
return new ContinuousIcebergEnumerator(
enumContext, assigner, scanContext, splitPlanner, enumState);
} else {
List<IcebergSourceSplit> splits = planSplitsForBatch(planningThreadName());
assigner.onDiscoveredSplits(splits);
return new StaticIcebergEnumerator(
enumContext, assigner, lazyTable(), scanContext, enumState);
}
7.RowDataReaderFunction
对于SQL语句这条线,Reader使用的读取函数是RowDataReaderFunction,相关的列信息对应的成员是readSchema,这个如果getProjectedSchema有就是用getProjectedSchema,如果没有就是前面的table.schema(),也就是表的所有列
readSchema方法就是完成上面所讲的选择的
DataIteratorReaderFunction是RowDataReaderFunction的父类,定义了基本特性
- apply
核心方法,主要是创建DataIterator并设置DataIteratorBatcher
public CloseableIterator<RecordsWithSplitIds<RecordAndPosition<T>>> apply(
IcebergSourceSplit split) {
DataIterator<T> inputIterator = createDataIterator(split);
inputIterator.seek(split.fileOffset(), split.recordOffset());
return batcher.batch(split.splitId(), inputIterator);
}
- DataIteratorReaderFunction
构造函数,由子类调用,会传入DataIteratorBatcher
public DataIteratorReaderFunction(DataIteratorBatcher<T> batcher) {
this.batcher = batcher;
}
RowDataReaderFunction传入父类的是ArrayPoolDataIteratorBatcher
super(
new ArrayPoolDataIteratorBatcher<>(
config,
new RowDataRecordFactory(
FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema)))));
- createDataIterator
这个由子类实现,用于创建DataIterator
public DataIterator<RowData> createDataIterator(IcebergSourceSplit split) {
return new DataIterator<>(
new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive),
split.task(),
io,
encryption);
}
8.数据流转
整体结构如下,最终数据输出点在右侧的IcebergSourceRecordEmitter,数据读取点在RowDataFileScanTaskReader当中创建的Reader读取器


最终的实际数据读取是在RowDataFileScanTaskReader当中根据不同的数据格式创建的对应的阅读器(在newIterable方法当中),以Parquet举例
在Iceberg的Parquet类的build当中创建实际的读取器,批流分别如下(还有没有传入Func的场景会走其他的创建流程)
if (batchedReaderFunc != null) {
return new VectorizedParquetReader<>(
file,
schema,
options,
batchedReaderFunc,
nameMapping,
filter,
reuseContainers,
caseSensitive,
maxRecordsPerBatch);
} else {
return new org.apache.iceberg.parquet.ParquetReader<>(
file,
schema,
options,
readerFunc,
nameMapping,
filter,
reuseContainers,
caseSensitive);
}
以ParquetReader为例,调用next接口,最终在advance
private void advance() {
while (shouldSkip[nextRowGroup]) {
nextRowGroup += 1;
reader.skipNextRowGroup();
}
PageReadStore pages;
try {
pages = reader.readNextRowGroup();
} catch (IOException e) {
throw new RuntimeIOException(e);
}
long rowPosition = rowGroupsStartRowPos[nextRowGroup];
nextRowGroupStart += pages.getRowCount();
nextRowGroup += 1;
model.setPageSource(pages, rowPosition);
}
9.关于嵌套列的关注点
对于嵌套列,IcebergTableSource中有两个核心的地方:applyProjection里是Iceberg处理流程里嵌套列的来源; getProjectedSchema是构建Source使用的列信息的来源
9.1.getProjectedSchema
方法内容前面已经帖出过,这里就是根据applyProjection的内容获取schema对应的列信息构建新的TableSchema,这里要关注schema的结构
schema的来源在FlinkDynamicTableFactory的createDynamicTableSource中
TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema());
这里传入的参数实际是Flink获取的完整的Table schema信息
9.2.FlinkSchemaUtil.convert
这里是做Flink Schema和Iceberg Schema的转换,注意嵌套列在两者的表现形式
icebergSchema,在IcebergSource的build方法里通过table.schema()获取
10.Flink 复杂列对象-ROW、RowType
ARRAY
MULTISET
MAP
ROW
RAW
Structured types Only exposed in user-defined functions yet.
上面是复杂列对象,但实际会形成嵌套列的应该只有ROW,此外在用户自定义方法中,应该还有Structured类型,先不考虑。
779

被折叠的 条评论
为什么被折叠?



