1.构建阶段——Flink为例
1.1.Flink流程构建
与写入过程类似,IcebergTableSource对标IcebergTableSink,produceDataStream对标consumeDataStream,在Flink SQL的同一处理流程中产生
这里稍微特殊的是,由于Flink的Source提供了两版实现,所以produceDataStream也根据条件内部有两条分支。FLIP27是Flink新的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);
}
}
以传统的DataStream追踪,通过FlinkSource的build构建,分批和流两种场景
public DataStream<RowData> build() {
Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null");
FlinkInputFormat format = buildFormat();
ScanContext context = contextBuilder.build();
TypeInformation<RowData> typeInfo =
FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project()));
if (!context.isStreaming()) {
int parallelism =
SourceUtil.inferParallelism(
readableConfig,
context.limit(),
() -> {
try {
return format.createInputSplits(0).length;
} catch (IOException e) {
throw new UncheckedIOException(
"Failed to create iceberg input splits for table: " + table, e);
}
});
if (env.getMaxParallelism() > 0) {
parallelism = Math.min(parallelism, env.getMaxParallelism());
}
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));
}
}
批和流的差异在于流有一个持续监听的工具类,负责持续的监听。两种方式使用的根本的Source源是一样的,都是FlinkInputFormat
以批来说,FlinkInputFormat会被Flink的处理流程封装成DataStreamSource
public <OUT> DataStreamSource<OUT> createInput(InputFormat<OUT, ?> inputFormat, TypeInformation<OUT> typeInfo) {
DataStreamSource source;
if (inputFormat instanceof FileInputFormat) {
FileInputFormat<OUT> format = (FileInputFormat)inputFormat;
source = this.createFileInput(format, typeInfo, "Custom File source", FileProcessingMode.PROCESS_ONCE, -1L);
} else {
source = this.createInput(inputFormat, typeInfo, "Custom Source");
}
return source;
}
1.2.FlinkInputFormat
FlinkInputFormat是Iceberg实现Flink的旧版本的Source的核心
FlinkInputFormat在FlinkSource.buildFormat,创建前核心两个步骤:1、通过tableLoader加载表信息,主要是Schema、FileIO、EncryptionManager(加解密表数据的模块);2、属性配置
通过加载表获取表信息
tableLoader.open();
try (TableLoader loader = tableLoader) {
table = loader.loadTable();
icebergSchema = table.schema();
io = table.io();
encryption = table.encryption();
tableLoader是在Flink创建DynamicTableSource的时候创建的,在FlinkDynamicTableFactory当中
if (catalog != null) {
tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath());
} else {
tableLoader =
createTableLoader(
catalogTable,
tableProps,
objectIdentifier.getDatabaseName(),
objectIdentifier.getObjectName());
}
return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration());
后面有两个配置设置,一是Flink的本地性,Flink的文件分配器可以利用本地性将本地文件分配给本地,这里是选择是否暴露主机信息以利用本地性。从逻辑上看,这一块目前仅适用于HDFS
public static final ConfigOption<Boolean> TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO =
ConfigOptions.key("table.exec.iceberg.expose-split-locality-info")
.booleanType()
.noDefaultValue()
.withDescription(
"Expose split host information to use Flink's locality aware split assigner.");
第二个是扫描元数据时使用的线程池大小,默认应该是CPU核数
public static final ConfigOption<Integer> TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE =
ConfigOptions.key("table.exec.iceberg.worker-pool-size")
.intType()
.defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE)
.withDescription("The size of workers pool used to plan or scan manifests.");
InputFormat的核心是创建了一个RowDataFileScanTaskReader,读文件最终功能都落在这个类上
this.rowDataReader =
new RowDataFileScanTaskReader(
tableSchema, context.project(), context.nameMapping(), context.caseSensitive());
2.基本流程
FlinkInputFormat是批模式读Iceberg的实现,核心四个过程,对应四个接口,分别实现数据分配和数据读取:1、createInputSplits;2、getInputSplitAssigner;3、open;4、nextRecord
createInputSplits,完成数据源的分片;getInputSplitAssigner是获取分配器,分配器内部方法将分片列表进行分配返回。这两个接口在JobManager执行
open和nextRecord基本一体的,在TaskManager执行。open负责构建迭代器,迭代器在分片上构建;nextRecord负责在迭代器的基础上读取数据
2.1.分片产生
整体思想是根据表的元数据信息,获取数据文件,然后构建数据分片,最终返回的是一个FlinkInputSplit队列,这个会作为后续读取数据的基础
// Called in Job manager, so it is OK to load table from catalog.
tableLoader.open();
final ExecutorService workerPool =
ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism());
try (TableLoader loader = tableLoader) {
Table table = loader.loadTable();
return FlinkSplitPlanner.planInputSplits(table, context, workerPool);
分片的核心是TableScan,批作业来说,最终实现在BaseTableScan.planTasks中进行ScanTask的构建。其中有一个ScanEvent的时间通知,但是没看到事件处理的地方
文件的获取基于Snapshot元数据文件,在DataTableScan.doPlanFiles中完成元数据文件的读取和解析,获取table对应的数据文件列表,最终在ManifestGroup.createFileScanTasks当中创建针对一个文件的扫描器(由于使用了大量的函数式编程,很多接口并不是在代码的位置调用的,而是延迟到后续方法调用时调用接口)
return CloseableIterable.transform(
entries,
entry -> {
DataFile dataFile = entry.file().copy(ctx.shouldKeepStats());
DeleteFile[] deleteFiles = ctx.deletes().forEntry(entry);
for (DeleteFile deleteFile : deleteFiles) {
ctx.scanMetrics().totalDeleteFileSizeInBytes().increment(deleteFile.fileSizeInBytes());
}
ctx.scanMetrics().totalFileSizeInBytes().increment(dataFile.fileSizeInBytes());
ctx.scanMetrics().resultDataFiles().increment();
ctx.scanMetrics().resultDeleteFiles().increment((long) deleteFiles.length);
return new BaseFileScanTask(
dataFile, deleteFiles, ctx.schemaAsString(), ctx.specAsString(), ctx.residuals());
});
2.2.读取数据
基于Flink的Source使用结构,读数据调用FlinkInputFormat的nextRecord接口
public RowData nextRecord(RowData reuse) {
currentReadCount++;
return iterator.next();
}
读数据基于迭代器,迭代器基于RowDataFileScanTaskReader构建
public void open(FlinkInputSplit split) {
this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption);
}
迭代器的数据来源于两个内容:1、FileScanTask;2、RowDataFileScanTaskReader。FileScanTask确定需要读取的具体文件,RowDataFileScanTaskReader负责实际读取数据
private CloseableIterator<T> openTaskIterator(FileScanTask scanTask) {
return fileScanTaskReader.open(scanTask, inputFilesDecryptor);
}
RowDataFileScanTaskReader当中会构建一个迭代器,构建方式如下
FlinkDeleteFilter deletes =
new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor);
CloseableIterable<RowData> iterable =
deletes.filter(
newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor));
task就是传入的FileScanTask,这里newIterable会根据文件存储类型创建对应的迭代器。(这里有疑问,获取的就是data文件,不确定这个DataTask指什么)
if (task.isDataTask()) {
throw new UnsupportedOperationException("Cannot read data task.");
} else {
switch (task.file().format()) {
case PARQUET:
iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor);
break;
最后就是调用这些数据格式的迭代器的next方法获取数据
以Parquet为例,Iceberg封装的最后一层迭代器就是ParquetIterable,内部有iterator方法返回ParquetIterator,ParquetIterator封装了实际的Parquet读数据,这里就进入了第三方Parquet的类——ParquetReader
private T advance() {
// this must be called in hasNext because it reuses an UnsafeRow
try {
T nextRecord = parquet.read();
this.needsAdvance = false;
this.hasNext = nextRecord != null;
return nextRecord;
} catch (IOException e) {
throw new RuntimeIOException(e);
}
}
3.接口
3.1.分片
Flink的分片流程走到FlinkSplitPlanner.planTasks,接下来的内部调用都是调用到通用的Iceberg接口,核心是TableScan。此处,根据流和批的不同,分别进行了IncrementalAppendScan和TableScan的实现
以批来说,调用的是BaseTableScan.planTasks
public CloseableIterable<CombinedScanTask> planTasks() {
CloseableIterable<FileScanTask> fileScanTasks = planFiles();
CloseableIterable<FileScanTask> splitFiles =
TableScanUtil.splitFiles(fileScanTasks, targetSplitSize());
return TableScanUtil.planTasks(
splitFiles, targetSplitSize(), splitLookback(), splitOpenFileCost());
}
3.1.1.planFiles
扫描文件的实现是其中对doPlanFiles接口的调用,这里的下层实现类使用的是DataTableScan
整个接口的过程就是按元数据的组织形式,从上往下读取,最终获取文件列表
首先是获取Snapshot
Snapshot snapshot = snapshot();
之后获取Manifest
List<ManifestFile> dataManifests = snapshot.dataManifests(io);
List<ManifestFile> deleteManifests = snapshot.deleteManifests(io);
基于Manifest,构建manifestGroup
ManifestGroup manifestGroup =
new ManifestGroup(io, dataManifests, deleteManifests)
.caseSensitive(isCaseSensitive())
.select(scanColumns())
.filterData(filter())
.specsById(table().specs())
.scanMetrics(scanMetrics())
.ignoreDeleted();
最终基于manifestGroup去获取文件,调用其planFiles接口
public CloseableIterable<FileScanTask> planFiles() {
return plan(ManifestGroup::createFileScanTasks);
}
createFileScanTasks接口逻辑如下,就是获取数据文件
private static CloseableIterable<FileScanTask> createFileScanTasks(
CloseableIterable<ManifestEntry<DataFile>> entries, TaskContext ctx) {
return CloseableIterable.transform(
entries,
entry -> {
DataFile dataFile = entry.file().copy(ctx.shouldKeepStats());
DeleteFile[] deleteFiles = ctx.deletes().forEntry(entry);
for (DeleteFile deleteFile : deleteFiles) {
ctx.scanMetrics().totalDeleteFileSizeInBytes().increment(deleteFile.fileSizeInBytes());
}
ctx.scanMetrics().totalFileSizeInBytes().increment(dataFile.fileSizeInBytes());
ctx.scanMetrics().resultDataFiles().increment();
ctx.scanMetrics().resultDeleteFiles().increment((long) deleteFiles.length);
return new BaseFileScanTask(
dataFile, deleteFiles, ctx.schemaAsString(), ctx.specAsString(), ctx.residuals());
});
}
createFileScanTasks的参数来自plan接口内的调用
Iterable<CloseableIterable<T>> tasks =
entries(
(manifest, entries) -> {
int specId = manifest.partitionSpecId();
TaskContext taskContext = taskContextCache.get(specId);
return createTasksFunc.apply(entries, taskContext);
});
再看entries接口,核心就是通过Manifest文件获取数据文件
public CloseableIterator<T> iterator() {
ManifestReader<DataFile> reader =
ManifestFiles.read(manifest, io, specsById)
.filterRows(dataFilter)
.filterPartitions(partitionFilter)
.caseSensitive(caseSensitive)
.select(columns)
.scanMetrics(scanMetrics);
CloseableIterable<ManifestEntry<DataFile>> entries;
if (ignoreDeleted) {
entries = reader.liveEntries();
} else {
entries = reader.entries();
}
ManifestReader读取文件,最终走的还是AVRO的读取类
switch (format) {
case AVRO:
AvroIterable<ManifestEntry<F>> reader =
Avro.read(file)
.project(ManifestEntry.wrapFileSchema(Types.StructType.of(fields)))
.rename("manifest_entry", GenericManifestEntry.class.getName())
.rename("partition", PartitionData.class.getName())
.rename("r102", PartitionData.class.getName())
.rename("data_file", content.fileClass())
.rename("r2", content.fileClass())
.classLoader(GenericManifestEntry.class.getClassLoader())
.reuseContainers()
.build();
3.1.2.splitFiles
上一个接口获取的是文件级别的列表,这一步操作是对文件进一步划分,按照指定的大小切分文件块,核心就是切分文件块的大小设置,可配
配置项为read.split.target-size,默认值为128 * 1024 * 1024; // 128 MB,对标的应该是HDFS的块大小,所以HDFS的块大小变更的时候,这里应该同步变更
public long targetSplitSize() {
long tableValue =
ops.current()
.propertyAsLong(TableProperties.SPLIT_SIZE, TableProperties.SPLIT_SIZE_DEFAULT);
return PropertyUtil.propertyAsLong(context.options(), TableProperties.SPLIT_SIZE, tableValue);
}
上一个接口构建的是BaseFileScanTask,这个接口没有实现split接口,它的子类SplitScanTask实现了,但是显示已经切分,不支持继续切分
BaseFileScanTask的父类BaseContentScanTask也实现了这个接口,BaseFileScanTask应该是用的父类的接口进行的划分
public Iterable<ThisT> split(long targetSplitSize) {
if (file.format().isSplittable()) {
if (file.splitOffsets() != null && OFFSET_ORDERING.isOrdered(file.splitOffsets())) {
return () ->
new OffsetsAwareSplitScanTaskIterator<>(
self(), length(), file.splitOffsets(), this::newSplitTask);
} else {
return () ->
new FixedSizeSplitScanTaskIterator<>(
self(), length(), targetSplitSize, this::newSplitTask);
}
}
return ImmutableList.of(self());
}
OffsetsAwareSplitScanTaskIterator是基于偏移量进行划分的实现,比如Parquet的row group offsets
FixedSizeSplitScanTaskIterator就是对于没有offset的使用基本的大小划分
3.1.3.planTasks
这个接口主要做的事就是把FileScanTask封装成CombinedScanTask
接口中对DeleteFile有特殊的均衡处理,这是一个Function,会被后续调用
// Check the size of delete file as well to avoid unbalanced bin-packing
Function<FileScanTask, Long> weightFunc =
file ->
Math.max(
file.length()
+ file.deletes().stream().mapToLong(ContentFile::fileSizeInBytes).sum(),
(1 + file.deletes().size()) * openFileCost);
这一块的合并就是根据目标大小将几个FileScanTask合并成一个。这里面是函数式的调用,最终的分片获取是在BinPacking.next,里面有FileScanTask的合并过程
void add(T item, long weight) {
this.binWeight += weight;
items.add(item);
}
3.1.4.文件过滤
在基于Manifest层获取数据文件时,会有一层过滤,读取文件的元数据根据过滤添加选择是否真的到读取数据文件
首先是过滤条件的来源,过滤条件来源于Flink SQL的条件,在构建DataStream时传入,如
IcebergTableSource.createDataStream
private DataStream<RowData> createDataStream(StreamExecutionEnvironment execEnv) {
return FlinkSource.forRowData()
.env(execEnv)
.tableLoader(loader)
.properties(properties)
.project(getProjectedSchema())
.limit(limit)
.filters(filters)
.flinkConf(readableConfig)
.build();
}
这个条件最终传入到ScanContext,在初始化FlinkInputFormat时传入,然后在doPlanFiles构建ManifestGroup时传入filter
ManifestGroup manifestGroup =
new ManifestGroup(io, dataManifests, deleteManifests)
.caseSensitive(isCaseSensitive())
.select(scanColumns())
.filterData(filter())
.specsById(table().specs())
.scanMetrics(scanMetrics())
.ignoreDeleted();
在ManifestReader的entries接口当中,在最后的metricsEvaluator.eval这一步,最终调用到InclusiveMetricsEvaluator的eval当中。在这一步当中,把数据文件的一些元数据信息读了出来,比如最大最小值
this.valueCounts = file.valueCounts();
this.nullCounts = file.nullValueCounts();
this.nanCounts = file.nanValueCounts();
this.lowerBounds = file.lowerBounds();
this.upperBounds = file.upperBounds();
然后在同接口的最后一步,通过访问者模式,基于过滤条件对文件进行了过滤
return ExpressionVisitors.visitEvaluator(expr, this);
最终调用到predicate接口,接口中根据过滤条件的条件类型(大于小于等于之类的)走不同的分支
case EQ:
return eq((BoundReference<T>) pred.term(), literalPred.literal());
最终走到InclusiveMetricsEvaluator的eq接口,在接口中使用之前获取的文件元数据(最大最小值之类的)与条件语句做过滤,最终根据返回条件选择是否要读取这个数据文件
public <T> Boolean eq(BoundReference<T> ref, Literal<T> lit) {
Integer id = ref.fieldId();
if (containsNullsOnly(id) || containsNaNsOnly(id)) {
return ROWS_CANNOT_MATCH;
}
if (lowerBounds != null && lowerBounds.containsKey(id)) {
T lower = Conversions.fromByteBuffer(ref.type(), lowerBounds.get(id));
if (NaNUtil.isNaN(lower)) {
// NaN indicates unreliable bounds. See the InclusiveMetricsEvaluator docs for more.
return ROWS_MIGHT_MATCH;
}
int cmp = lit.comparator().compare(lower, lit.value());
if (cmp > 0) {
return ROWS_CANNOT_MATCH;
}
}
if (upperBounds != null && upperBounds.containsKey(id)) {
T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id));
int cmp = lit.comparator().compare(upper, lit.value());
if (cmp < 0) {
return ROWS_CANNOT_MATCH;
}
}
return ROWS_MIGHT_MATCH;
}
3.2.数据读取
3.2.1.DataIterator构建
Flink的处理中,读取数据就是通过迭代器不断的调用next,数据读取的基础就是迭代器的构建
public RowData nextRecord(RowData reuse) {
currentReadCount++;
return iterator.next();
}
迭代器的构建在open接口中
public void open(FlinkInputSplit split) {
this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption);
}
split就是前面的分片,核心的一个成员是读取器rowDataReader,其创建在FlinkInputFormat的初始化中
this.rowDataReader =
new RowDataFileScanTaskReader(
tableSchema, context.project(), context.nameMapping(), context.caseSensitive());
RowDataFileScanTaskReader目前是Flink的一个封装类,内部封装了Iceberg方法,自定义实现可以参考这个类的实现
3.2.2.文件迭代器
读数据就是不断从迭代器next获取数据的过程,关键点在于文件的读取,这里就是一层一层迭代器的封装
在Flink的接口定义中,构建迭代器时传入了一个分片,但是一个分片可能包含多个文件,读取数据时对每个文件都会构建一个迭代器读取,核心就是上节的RowDataFileScanTaskReader,其open方法返回一个迭代器,由上层next方法调用返回数据
private CloseableIterator<T> openTaskIterator(FileScanTask scanTask) {
return fileScanTaskReader.open(scanTask, inputFilesDecryptor);
}
在open方法中,核心还是创建迭代器,在newIterable接口中,其中根据文件格式,创建不同的文件迭代器(这里一个待研究的就是isDataTask的定义)
if (task.isDataTask()) {
throw new UnsupportedOperationException("Cannot read data task.");
} else {
switch (task.file().format()) {
case PARQUET:
iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor);
break;
case AVRO:
iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor);
break;
case ORC:
iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor);
break;
default:
throw new UnsupportedOperationException(
"Cannot read unknown format: " + task.file().format());
}
}
以Parquet为例,构建时调用Iceberg的Parquet封装类
Parquet.ReadBuilder builder =
Parquet.read(inputFilesDecryptor.getInputFile(task))
.split(task.start(), task.length())
.project(schema)
.createReaderFunc(
fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant))
.filter(task.residual())
.caseSensitive(caseSensitive)
.reuseContainers();
来到Iceberg封装的Parquet类的build方法中,这里有个大分支,是否设置了读取函数,Flink这边的定义是传入了读取函数的,所以走第一条分支构建
接口分支内主要就是做了一些参数设置,然后最重要的就是创建Parquet的读取类,这个类还是Iceberg封装的,没到第三方的包
类有两个,一个用于批一个用于流,Flink这边用的是流的接口
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);
}
3.2.3.数据读取
数据读取就是一路调用next,最终来到上节定义的ParquetReader中,这里构建了一个FileIterator的迭代器
public CloseableIterator<T> iterator() {
FileIterator<T> iter = new FileIterator<>(init());
addCloseable(iter);
return iter;
}
next方法最终调用的就是FileIterator的next方法,与写入类似,基于model调用
if (reuseContainers) {
this.last = model.read(last);
} else {
this.last = model.read(null);
}
model的定义在于前面的init方法,定义了一个ReadConf
ReadConf<T> readConf =
new ReadConf<>(
input,
options,
expectedSchema,
filter,
readerFunc,
null,
nameMapping,
reuseContainers,
caseSensitive,
null);
与写入类似,这里基于readerFunc创建model
if (readerFunc != null) {
this.model = (ParquetValueReader<T>) readerFunc.apply(typeWithIds);
this.vectorizedModel = null;
this.columnChunkMetaDataForRowGroups = null;
} else {
this.model = null;
this.vectorizedModel = (VectorizedReader<T>) batchedReaderFunc.apply(typeWithIds);
this.columnChunkMetaDataForRowGroups = getColumnChunkMetadataForRowGroups();
}
接下来的过程与写数据的流程非常类似了,只是所有的写换成读
首先是Function,这里定义的是FlinkParquetReaders,然后初始化定义了一个visit的模式,这个Function就是自定义实现要做的最重要的一个类
public static ParquetValueReader<RowData> buildReader(
Schema expectedSchema, MessageType fileSchema, Map<Integer, ?> idToConstant) {
return (ParquetValueReader<RowData>)
TypeWithSchemaVisitor.visit(
expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant));
}
同样的基于访问者模式,这边根据数据类型,有不同的分支,以基本类型来说
} else if (type.isPrimitive()) {
org.apache.iceberg.types.Type.PrimitiveType iPrimitive =
iType != null ? iType.asPrimitiveType() : null;
return visitor.primitive(iPrimitive, type.asPrimitiveType());
} else {
一样的流程,根据数据类型,构建不同的读取实现类
if (primitive.getOriginalType() != null) {
switch (primitive.getOriginalType()) {
case ENUM:
case JSON:
case UTF8:
return new StringReader(desc);
case INT_8:
case INT_16:
case INT_32:
if (expected.typeId() == Types.LongType.get().typeId()) {
return new ParquetValueReaders.IntAsLongReader(desc);
} else {
return new ParquetValueReaders.UnboxedReader<>(desc);
读数据就是基于这里的实现类,以StringReader来说
public StringData read(StringData ignored) {
Binary binary = column.nextBinary();
ByteBuffer buffer = binary.toByteBuffer();
if (buffer.hasArray()) {
return StringData.fromBytes(
buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining());
} else {
return StringData.fromBytes(binary.getBytes());
}
}
}
接下来,column这里Iceberg还进行了两层封装,先是ColumnIterator
public Binary nextBinary() {
this.triplesRead += 1;
advance();
return pageIterator.nextBinary();
}
这里最重要的是advance中对pageIterator的处理,设置一个page
DataPage page = pageSource.readPage();
if (page != null) {
pageIterator.setPage(page);
this.advanceNextPageCount += pageIterator.currentPageCount();
} else {
pageSource是一个PageReader,其定义也是来自于前面的ReadConf,ParquetFileReader是第三方的Parquet包的类
private static ParquetFileReader newReader(InputFile file, ParquetReadOptions options) {
try {
return ParquetFileReader.open(ParquetIO.file(file), options);
} catch (IOException e) {
throw new RuntimeIOException(e, "Failed to open Parquet file: %s", file.location());
}
}
下一层就是PageIterator
public Binary nextBinary() {
advance();
try {
return values.readBytes();
} catch (RuntimeException e) {
throw handleRuntimeException(e);
}
}
values的定义为,这里就是第三方Parquet的包了
if (dataEncoding.usesDictionary()) {
if (dictionary == null) {
throw new ParquetDecodingException(
"could not read page in col "
+ desc
+ " as the dictionary was missing for encoding "
+ dataEncoding);
}
this.values =
dataEncoding.getDictionaryBasedValuesReader(desc, ValuesType.VALUES, dictionary);
} else {
this.values = dataEncoding.getValuesReader(desc, ValuesType.VALUES);
}
3.2.4.补充–字典编码
上节最后的分支是基于是否使用字典编码产生的分支(字典太大会回退成普通编码)
Dictionary Encoding,使用字典压缩来存储数据,当大量重复值时,可以节省大量内存,其本质是一种类似指针的东西
Parquet中每一个列块会有一个专门的dictionary page来存储字典
整体的编码思想如下,假设有个字段name,在10条数据中的值分别为:
name:
bruce, cake, bruce, kevin, bruce, kevin, cake, leo, cake, bruce
对其字典编码可以存储成如下的数据,这种结构也可用于谓词下推的快速过滤:
name:
0, 1, 0, 2, 0, 2, 1, 3, 1, 0
dictionary:
0 -> bruce, 1 -> cake, 2 -> kevin, 3 -> leo
针对字典的两层结构,还会做处理,dictionary page就用普通的编码,具体的数据存储这一块的所有数字类型的数据还会采用一层编码,Run Length Encoding / Bit-Packing Hybrid,就是RLE和Bit-Packing的混用
Run Length Encoding,运行长度编码,就是记录重复数,适合连续重复数字
1 1 2 2 2 2 1 3 3 3 3 3
2-1 4-2 1-1 5-3
Bit-Packing的基础就是不是所有的int32/int64的数值都需要完整的32/64位,核心思想就是在一个32/64位当中插入多个数值,利用特殊的方法来识别(位运算)
Parquet官网介绍还有其他的编码格式,未具体研究