openinx commented on a change in pull request #1293: URL: https://github.com/apache/iceberg/pull/1293#discussion_r469955623
########## 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())); Review comment: Q: Are we binding the iceberg table's hadoop configuration with flink job manager's hadoop configuration ? Is it possible to access an iceberg table in the hadoop cluster which is different with flink's hadoop cluster ? Seems a more reasonable way is: Passing a customized `SerializeableConfiguration` from client, then job manager could access any hadoop clusters. ---------------------------------------------------------------- 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]
