openinx commented on a change in pull request #1293: URL: https://github.com/apache/iceberg/pull/1293#discussion_r469893630
########## File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java ########## @@ -0,0 +1,224 @@ +/* + * 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.source; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; + +/** + * Flink Iceberg {@link InputFormat}. + * - Calls that occur on the Job manager side: {@link #createInputSplits} and {@link #getInputSplitAssigner}. + * - Calls that occur on the Task side: {@link #open}, {@link #reachedEnd}, {@link #nextRecord} and {@link #close}. + */ +public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> { + + private final TableLoader tableLoader; + private final Schema tableSchema; + private final List<String> projectedFields; + private final ScanOptions options; + private final List<Expression> filterExpressions; + private final FileIO io; + private final EncryptionManager encryption; + + private transient RowDataReader reader; + private transient boolean fetched; + private transient boolean hasNext; + + private FlinkInputFormat( + TableLoader tableLoader, Schema tableSchema, FileIO io, EncryptionManager encryption, + List<String> projectedFields, List<Expression> filterExpressions, ScanOptions options) { + this.tableLoader = tableLoader; + this.tableSchema = tableSchema; + this.projectedFields = projectedFields; + this.options = options; + this.filterExpressions = filterExpressions; + this.io = io; + this.encryption = encryption; + } + + @VisibleForTesting + Schema getTableSchema() { + return tableSchema; + } + + @VisibleForTesting + List<String> getProjectedFields() { + return projectedFields; + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { + // Legacy method, not be used. + return null; + } + + @Override + public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { + // Invoked by Job manager, so it is OK to load table from catalog. + tableLoader.open(HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration())); + try (TableLoader loader = tableLoader) { + Table table = loader.loadTable(); + Schema expectedSchema = FlinkSchemaUtil.pruneWithoutReordering(tableSchema, projectedFields); + FlinkSplitGenerator generator = new FlinkSplitGenerator(table, expectedSchema, options, filterExpressions); + return generator.createInputSplits(); + } + } + + @Override + public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { + return new DefaultInputSplitAssigner(inputSplits); + } + + @Override + public void configure(Configuration parameters) { + } + + @Override + public void open(FlinkInputSplit split) { + String nameMappingString = options.getNameMapping(); + Schema expectedSchema = FlinkSchemaUtil.pruneWithoutReordering(tableSchema, projectedFields); + List<String> expectedNameList = expectedSchema.asStruct().fields().stream() + .map(Types.NestedField::name) + .collect(Collectors.toList()); + int[] finalProjection = projectedFields == null ? null : + projectedFields.stream().mapToInt(expectedNameList::indexOf).toArray(); + this.reader = new RowDataReader(split.getTask(), io, encryption, finalProjection, expectedSchema, + nameMappingString); + } + + @Override + public boolean reachedEnd() throws IOException { + if (!fetched) { + hasNext = reader.next(); + fetched = true; + } + return !hasNext; + } + + @Override + public RowData nextRecord(RowData reuse) throws IOException { + if (reachedEnd()) { + return null; + } + + fetched = false; + return reader.get(); + } + + @Override + public void close() throws IOException { + if (reader != null) { + reader.close(); + } + } + + public static Builder builder() { + return new Builder(); + } + + public static final class Builder { + private TableLoader tableLoader; + private Schema tableSchema; + private List<String> projectedFields; + private ScanOptions options = ScanOptions.builder().build(); + private List<Expression> filterExpressions = Lists.newArrayList(); + private FileIO io; + private EncryptionManager encryption; + + private Builder() {} + + // -------------------------- Required options ------------------------------- + + public Builder table(Table newTable) { + this.tableSchema = newTable.schema(); + this.io = newTable.io(); + this.encryption = newTable.encryption(); + return this; + } + + public Builder tableLoader(TableLoader newLoader) { + this.tableLoader = newLoader; + return this; + } + + // -------------------------- Optional options ------------------------------- + + public Builder filters(List<Expression> newFilters) { + this.filterExpressions = newFilters; + return this; + } + + public Builder projectedFields(List<String> newProjectedFields) { + this.projectedFields = newProjectedFields; + return this; + } + + public Builder options(ScanOptions newOptions) { + this.options = newOptions; + return this; + } + + public Builder schema(Schema newSchema) { + this.tableSchema = newSchema; + return this; + } + + public Builder io(FileIO newIO) { + this.io = newIO; + return this; + } + + public Builder encryption(EncryptionManager newEncryption) { + this.encryption = newEncryption; + return this; + } + + public FlinkInputFormat build() { + return new FlinkInputFormat( Review comment: We'd better to have a `Precondition#check` for those arguments in case of throw NPE in the following call stack. ########## File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java ########## @@ -0,0 +1,109 @@ +/* + * 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.source; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.flink.types.Row; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * Test {@link FlinkInputFormat}. + */ +public class TestFlinkInputFormat extends TestFlinkScan { + + private FlinkInputFormat.Builder builder; + + public TestFlinkInputFormat(String fileFormat) { + super(fileFormat); + } + + @Override + public void before() throws IOException { + super.before(); + builder = FlinkInputFormat.builder().tableLoader(TableLoader.fromHadoopTable(warehouse)); + } + + @Override + protected List<Row> execute(Table table, List<String> projectFields) throws IOException { + return run(builder.table(table).projectedFields(projectFields).build()); + } + + @Override + protected List<Row> executeWithSnapshotId(Table table, long snapshotId) throws IOException { + return run(builder.table(table).options(ScanOptions.builder().snapshotId(snapshotId).build()).build()); Review comment: Should we also add unit tests for following cases: 1. scan with both `startSnapshotId` and `endSnapshotId`; 2. scan with only `asOfTimestamp`; 3. scan with only `startSnapshotId` . ########## File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java ########## @@ -98,4 +102,24 @@ public static TableSchema toSchema(RowType rowType) { } return builder.build(); } + + /** + * Prune columns from a {@link Schema} using a projected fields. + * TODO Why Spark care about filters? Review comment: As the [javadoc](https://github.com/apache/iceberg/blob/ef801726bd6627ddcda1bd238894ea8ccdae5f39/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java#L193) said: > The filters list of {@link Expression} is used to ensure that columns referenced by filters are projected. For my understanding, when doing filter push down, we need to keep the columns which has been involved in push-down filter even if it does not in the projection column list. Now we do not implement the `FilterableTableSource` interface, so I think we don't need to consider it now. ########## File path: flink/src/main/java/org/apache/iceberg/flink/source/RowDataReader.java ########## @@ -0,0 +1,109 @@ +/* + * 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.source; + +import java.util.Map; +import java.util.Set; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.data.FlinkAvroReader; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; + +class RowDataReader extends BaseDataReader<RowData> { + + private final int[] finalProjection; + private final Schema expectedSchema; + private final String nameMapping; + + RowDataReader(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption, int[] finalProjection, + Schema expectedSchema, String nameMapping) { + super(task, fileIo, encryption); + this.finalProjection = finalProjection; + this.expectedSchema = expectedSchema; + this.nameMapping = nameMapping; + } + + @Override + protected CloseableIterator<RowData> nextTaskIterator(FileScanTask task) { + // schema or rows returned by readers + PartitionSpec spec = task.spec(); + Set<Integer> idColumns = spec.identitySourceIds(); + Schema partitionSchema = TypeUtil.select(expectedSchema, idColumns); + + Map<Integer, ?> idToConstant = partitionSchema.columns().isEmpty() ? ImmutableMap.of() : + PartitionUtil.constantsMap(task, RowDataReader::convertConstant); + CloseableIterable<RowData> iterable = newIterable(task, idToConstant); + ProjectionRowData projectionRow = new ProjectionRowData(); + return (finalProjection == null ? iterable : CloseableIterable.transform( + iterable, rowData -> (RowData) projectionRow.replace(rowData, finalProjection))).iterator(); Review comment: Q: Why we need to transform the `CloseableIterable<RowData>` to be a projected `RowData` iterable again, I mean we've created an AVRO iterable with the projected read schema, it should guarantee that iterable only contains the projected columns ? ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
