Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,58 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.flink.data;

import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.avro.AvroFormatModel;
import org.apache.iceberg.formats.FormatModelRegistry;
import org.apache.iceberg.orc.ORCFormatModel;
import org.apache.iceberg.parquet.ParquetFormatModel;

public class FlinkFormatModels {
public static void register() {
FormatModelRegistry.register(
ParquetFormatModel.create(
RowData.class,
RowType.class,
(icebergSchema, fileSchema, engineSchema) ->
FlinkParquetWriters.buildWriter(engineSchema, fileSchema),
(icebergSchema, fileSchema, engineSchema, idToConstant) ->
FlinkParquetReaders.buildReader(icebergSchema, fileSchema, idToConstant)));

FormatModelRegistry.register(
AvroFormatModel.create(
RowData.class,
RowType.class,
(icebergSchema, fileSchema, engineSchema) -> new FlinkAvroWriter(engineSchema),
(icebergSchema, fileSchema, engineSchema, idToConstant) ->
FlinkPlannedAvroReader.create(icebergSchema, idToConstant)));

FormatModelRegistry.register(
ORCFormatModel.create(
RowData.class,
RowType.class,
(icebergSchema, fileSchema, engineSchema) ->
FlinkOrcWriter.buildWriter(engineSchema, icebergSchema),
(icebergSchema, fileSchema, engineSchema, idToConstant) ->
new FlinkOrcReader(icebergSchema, fileSchema, idToConstant)));
}

private FlinkFormatModels() {}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,28 +25,19 @@
import java.io.Serializable;
import java.util.Map;
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.data.StringData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.Table;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.data.BaseFileWriterFactory;
import org.apache.iceberg.data.RegistryBasedFileWriterFactory;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.flink.data.FlinkAvroWriter;
import org.apache.iceberg.flink.data.FlinkOrcWriter;
import org.apache.iceberg.flink.data.FlinkParquetWriters;
import org.apache.iceberg.orc.ORC;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;

public class FlinkFileWriterFactory extends BaseFileWriterFactory<RowData> implements Serializable {
private RowType dataFlinkType;
private RowType equalityDeleteFlinkType;

private FlinkFileWriterFactory(
public class FlinkFileWriterFactory extends RegistryBasedFileWriterFactory<RowData, RowType>
implements Serializable {
FlinkFileWriterFactory(
Table table,
FileFormat dataFileFormat,
Schema dataSchema,
Expand All @@ -62,85 +53,30 @@ private FlinkFileWriterFactory(
super(
table,
dataFileFormat,
RowData.class,
dataSchema,
dataSortOrder,
deleteFileFormat,
equalityFieldIds,
equalityDeleteRowSchema,
equalityDeleteSortOrder,
writeProperties);

this.dataFlinkType = dataFlinkType;
this.equalityDeleteFlinkType = equalityDeleteFlinkType;
}

static Builder builderFor(Table table) {
return new Builder(table);
}

@Override
protected void configureDataWrite(Avro.DataWriteBuilder builder) {
builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType()));
}

@Override
protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) {
builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType()));
}

@Override
protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) {}

@Override
protected void configureDataWrite(Parquet.DataWriteBuilder builder) {
builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType));
}

@Override
protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) {
builder.createWriterFunc(
msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType));
}

@Override
protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) {
builder.transformPaths(path -> StringData.fromString(path.toString()));
}

@Override
protected void configureDataWrite(ORC.DataWriteBuilder builder) {
builder.createWriterFunc(
(iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema));
}

@Override
protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) {
builder.createWriterFunc(
(iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema));
writeProperties,
dataFlinkType == null ? FlinkSchemaUtil.convert(dataSchema) : dataFlinkType,
equalityDeleteInputSchema(equalityDeleteFlinkType, equalityDeleteRowSchema));
}

@Override
protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) {
builder.transformPaths(path -> StringData.fromString(path.toString()));
}

private RowType dataFlinkType() {
if (dataFlinkType == null) {
Preconditions.checkNotNull(dataSchema(), "Data schema must not be null");
this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema());
private static RowType equalityDeleteInputSchema(RowType rowType, Schema rowSchema) {
if (rowType != null) {
return rowType;
} else if (rowSchema != null) {
return FlinkSchemaUtil.convert(rowSchema);
} else {
return null;
}

return dataFlinkType;
}

private RowType equalityDeleteFlinkType() {
if (equalityDeleteFlinkType == null) {
Preconditions.checkNotNull(
equalityDeleteRowSchema(), "Equality delete schema must not be null");
this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema());
}

return equalityDeleteFlinkType;
static Builder builderFor(Table table) {
return new Builder(table);
}

public static class Builder {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,30 +24,23 @@
import org.apache.flink.table.data.RowData;
import org.apache.flink.table.types.logical.RowType;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.MetadataColumns;
import org.apache.iceberg.Schema;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.avro.Avro;
import org.apache.iceberg.data.DeleteFilter;
import org.apache.iceberg.encryption.InputFilesDecryptor;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.flink.FlinkSchemaUtil;
import org.apache.iceberg.flink.FlinkSourceFilter;
import org.apache.iceberg.flink.RowDataWrapper;
import org.apache.iceberg.flink.data.FlinkOrcReader;
import org.apache.iceberg.flink.data.FlinkParquetReaders;
import org.apache.iceberg.flink.data.FlinkPlannedAvroReader;
import org.apache.iceberg.flink.data.RowDataProjection;
import org.apache.iceberg.flink.data.RowDataUtil;
import org.apache.iceberg.formats.FormatModelRegistry;
import org.apache.iceberg.formats.ReadBuilder;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.CloseableIterator;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.mapping.NameMappingParser;
import org.apache.iceberg.orc.ORC;
import org.apache.iceberg.parquet.Parquet;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.types.TypeUtil;
import org.apache.iceberg.util.PartitionUtil;

@Internal
Expand All @@ -73,8 +66,7 @@ public RowDataFileScanTaskReader(
if (filters != null && !filters.isEmpty()) {
Expression combinedExpression =
filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and);
this.rowFilter =
new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive);
this.rowFilter = new FlinkSourceFilter(projectedSchema, combinedExpression, caseSensitive);
} else {
this.rowFilter = null;
}
Expand Down Expand Up @@ -112,23 +104,23 @@ private CloseableIterable<RowData> newIterable(
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());
ReadBuilder<RowData, RowType> builder =
FormatModelRegistry.readBuilder(
task.file().format(), RowData.class, inputFilesDecryptor.getInputFile(task));

if (nameMapping != null) {
builder.withNameMapping(NameMappingParser.fromJson(nameMapping));
}

iter =
builder
.project(schema)
.idToConstant(idToConstant)
.split(task.start(), task.length())
.caseSensitive(caseSensitive)
.filter(task.residual())
.reuseContainers()
.build();
}

if (rowFilter != null) {
Expand All @@ -137,72 +129,6 @@ private CloseableIterable<RowData> newIterable(
return iter;
}

private CloseableIterable<RowData> newAvroIterable(
FileScanTask task,
Schema schema,
Map<Integer, ?> idToConstant,
InputFilesDecryptor inputFilesDecryptor) {
Avro.ReadBuilder builder =
Avro.read(inputFilesDecryptor.getInputFile(task))
.reuseContainers()
.project(schema)
.split(task.start(), task.length())
.createReaderFunc(readSchema -> FlinkPlannedAvroReader.create(schema, idToConstant));

if (nameMapping != null) {
builder.withNameMapping(NameMappingParser.fromJson(nameMapping));
}

return builder.build();
}

private CloseableIterable<RowData> newParquetIterable(
FileScanTask task,
Schema schema,
Map<Integer, ?> idToConstant,
InputFilesDecryptor inputFilesDecryptor) {
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();

if (nameMapping != null) {
builder.withNameMapping(NameMappingParser.fromJson(nameMapping));
}

return builder.build();
}

private CloseableIterable<RowData> newOrcIterable(
FileScanTask task,
Schema schema,
Map<Integer, ?> idToConstant,
InputFilesDecryptor inputFilesDecryptor) {
Schema readSchemaWithoutConstantAndMetadataFields =
TypeUtil.selectNot(
schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds()));

ORC.ReadBuilder builder =
ORC.read(inputFilesDecryptor.getInputFile(task))
.project(readSchemaWithoutConstantAndMetadataFields)
.split(task.start(), task.length())
.createReaderFunc(
readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant))
.filter(task.residual())
.caseSensitive(caseSensitive);

if (nameMapping != null) {
builder.withNameMapping(NameMappingParser.fromJson(nameMapping));
}

return builder.build();
}

private static class FlinkDeleteFilter extends DeleteFilter<RowData> {
private final RowType requiredRowType;
private final RowDataWrapper asStructLike;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@
import org.apache.iceberg.Table;
import org.apache.iceberg.TableProperties;
import org.apache.iceberg.common.DynFields;
import org.apache.iceberg.data.BaseFileWriterFactory;
import org.apache.iceberg.data.RegistryBasedFileWriterFactory;
import org.apache.iceberg.flink.FlinkWriteConf;
import org.apache.iceberg.flink.FlinkWriteOptions;
import org.apache.iceberg.flink.SimpleDataUtil;
Expand Down Expand Up @@ -238,21 +238,21 @@ private static Map<String, String> appenderProperties(
testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1);

testHarness.prepareSnapshotPreBarrier(1L);
DynFields.BoundField<IcebergStreamWriter> operatorField =
DynFields.BoundField<IcebergStreamWriter<?>> operatorField =
DynFields.builder()
.hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator")
.build(testHarness.getOperatorFactory());
DynFields.BoundField<TaskWriter> writerField =
DynFields.BoundField<TaskWriter<?>> writerField =
DynFields.builder()
.hiddenImpl(IcebergStreamWriter.class, "writer")
.build(operatorField.get());
DynFields.BoundField<FileWriterFactory> writerFactoryField =
DynFields.BoundField<FileWriterFactory<?>> writerFactoryField =
DynFields.builder()
.hiddenImpl(BaseTaskWriter.class, "writerFactory")
.build(writerField.get());
DynFields.BoundField<Map<String, String>> propsField =
DynFields.builder()
.hiddenImpl(BaseFileWriterFactory.class, "writerProperties")
.hiddenImpl(RegistryBasedFileWriterFactory.class, "writerProperties")
.build(writerFactoryField.get());
return propsField.get();
}
Expand Down
Loading