You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/08/17 06:23:06 UTC

[GitHub] [iceberg] JingsongLi opened a new pull request #1346: Flink: Introduce Flink InputFormat

JingsongLi opened a new pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346


   This is subtask of #1293


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-689227758


   Is this one next to review?


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475309860



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -246,7 +256,7 @@ public void setBatchContext(long batchOffsetInFile) {
 
     StructReader(List<OrcValueReader<?>> readers, Types.StructType struct, Map<Integer, ?> idToConstant) {
       super(readers, struct, idToConstant);
-      this.numFields = readers.size();
+      this.numFields = struct.fields().size();

Review comment:
       This fixes Orc Reader (with partition) bug.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487611031



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {

Review comment:
       Nevermind, I see that this is an option after all.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472018847



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/RowDataIterator.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.List;
+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 RowDataIterator extends DataIterator<RowData> {
+
+  private final String nameMapping;
+
+  RowDataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption, Schema tableSchema,
+                  List<String> projectedFields, String nameMapping) {
+    super(task, fileIo, encryption, tableSchema, projectedFields);
+    this.nameMapping = nameMapping;
+  }
+
+  @Override
+  protected CloseableIterator<RowData> nextTaskIterator(FileScanTask task) {
+    // schema or rows returned by readers

Review comment:
       We can delete it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487603555



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       Is Flink using Java serialization across versions? That seems like a big risk to me. I'd prefer to only use Java serialization between processes running the exact same version of Iceberg. If we need to serialize across versions (like for checkpoint data) then I think we should worry about compatibility a lot more.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494662994



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       Field reordering tests are at the file format level. Each file format has to be able to project columns in the requested order. So any reordered schema should work as long as it is passed down correctly, which is what the new `convert` method does.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testProjection() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> inputRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords);
+    assertRows(execute(table, Collections.singletonList("data")), Row.of(inputRecords.get(0).get(0)));
+  }
+
+  @Test
+  public void testIdentityPartitionProjections() throws Exception {
+    Schema logSchema = new Schema(
+        Types.NestedField.optional(1, "id", Types.IntegerType.get()),
+        Types.NestedField.optional(2, "dt", Types.StringType.get()),
+        Types.NestedField.optional(3, "level", Types.StringType.get()),
+        Types.NestedField.optional(4, "message", Types.StringType.get())
+    );
+    PartitionSpec spec =
+        PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build();
+
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), logSchema, spec);
+    List<Record> inputRecords = RandomGenericData.generate(logSchema, 10, 0L);
+
+    int idx = 0;
+    AppendFiles append = table.newAppend();
+    for (Record record : inputRecords) {
+      record.set(1, "2020-03-2" + idx);
+      record.set(2, Integer.toString(idx));
+      append.appendFile(new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).writeFile(
+          org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), ImmutableList.of(record)));
+      idx += 1;
+    }
+    append.commit();
+
+    // individual fields
+    validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords);
+    // field pairs
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords);
+    // out-of-order pairs
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords);
+    // out-of-order triplets
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "level", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "dt", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "message", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "message", "dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "dt", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "level", "dt"), inputRecords);
+  }
+
+  private void validateIdentityPartitionProjections(Table table, List<String> projectedFields,
+      List<Record> inputRecords) throws IOException {
+    List<Row> rows = execute(table, projectedFields);
+
+    for (int pos = 0; pos < inputRecords.size(); pos++) {
+      Record inputRecord = inputRecords.get(pos);
+      Row actualRecord = rows.get(pos);
+
+      for (int i = 0; i < projectedFields.size(); i++) {
+        String name = projectedFields.get(i);
+        Assert.assertEquals(
+            "Projected field " + name + " should match", inputRecord.getField(name), actualRecord.getField(i));
+      }
+    }
+  }
+
+  @Test
+  public void testSnapshotReads() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+
+    GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER);
+
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    helper.appendToTable(expectedRecords);
+    long snapshotId = table.currentSnapshot().snapshotId();
+
+    long timestampMillis = table.currentSnapshot().timestampMillis();
+
+    // produce another timestamp
+    Thread.sleep(10);

Review comment:
       We usually introduce a spin to avoid sleeping for long durations in lots of tests, like this: https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java#L78

##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;

Review comment:
       I find it really strange that this is delegated to a subclass, given that it builds a very specific nested projection.
   
   Why not make this use a method like `execute(Table, List<String>)`, but pass in the projection instead of a list of fields?
   
   Then you could keep all of the schema details in the test method here, rather than delegating this assertion. I think it would be cleaner.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;

Review comment:
       Why is this implemented by the subclass? Couldn't this just call `assertRecords` directly?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488373628



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.LocatableInputSplit;
+import org.apache.iceberg.CombinedScanTask;
+
+/**
+ * TODO Implement {@link LocatableInputSplit}.
+ */
+public class FlinkInputSplit implements InputSplit {
+
+  private final int splitNumber;
+  private final CombinedScanTask task;
+
+  FlinkInputSplit(int splitNumber, CombinedScanTask task) {
+    this.splitNumber = splitNumber;
+    this.task = task;
+  }
+
+  @Override
+  public int getSplitNumber() {
+    return splitNumber;
+  }
+
+  CombinedScanTask getTask() {
+    return task;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    FlinkInputSplit that = (FlinkInputSplit) o;
+    return splitNumber == that.splitNumber;

Review comment:
       The `equals`, `hashCode` and `toString` are required by special assigner, but in `DefaultInputSplitAssigner`, we don't need them.
   I think I can remove them.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488379347



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();

Review comment:
       I think we remove this `select`, because what Flink SQL want, is an order changed select, instead of using original iceberg table order.
   We should provide a unified `select`, so I think we can provide in Flink side now, and we can use `project`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r473480966



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       You should be able to use the expected/projection schema. All readers should reorder columns to produce the requested column order.
   
   `AvroSchemaWithTypeVisitor` is used to traverse the file schema to create the reader structure, but that's because fields in Avro must be read in the file's order. But when that reader adds data columns to records, the values are put in the correct order because the [`ResolvingDecoder` returns the correct position](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java#L645-L648) in the projection schema.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476985164



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *

Review comment:
       Flink SQL did not have good support for nested projection before. I'll verify it and try to implement nested fields projection.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487611604



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       How is this different than an unbounded builder? I don't see anything that passes whether the stream should be bounded or unbounded. It seems like this should pass that information so that the input adapter can plan the current table scan, rather than checking for new data later.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494690595



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;

Review comment:
       Why is this implemented by the subclass? Couldn't this just call `assertRecords` directly?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r473581304



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       Thanks @openinx and @rdblue ! I'm very happy to be able to solve my confusion. I will do it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472011890



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       Continue with the question from [here](https://github.com/apache/iceberg/pull/1293#discussion_r469938063). If we could produce a  ordered & projected schema in this method (Saying if this method is `pruneWithReordering`), then seems we don't have to convert the read RowData to the correct order [here](https://github.com/apache/iceberg/pull/1346/files#diff-9e6ac35840fe0e9f8bacbe12c574c4eaR64) ? 
   
   I'd prefer to use the correct projected schema to read the target RowData if possible, rather than reading RowData in a disordered schema and then order them in an iterator transformation.  Because this is in the critical read path and an extra RowData transformation will cost more resources , also make the codes hard to follow. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472020381



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       > Another case: Project with a new renamed schema
   
   You mean: Create a table, insert some data, rename some fields, insert some data. Then read table using Flink?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475303761



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;

Review comment:
       > will we also need to put those writer related classes into sink package ?
   
   I think we can.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r486182970



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {
+          Utf8 utf8 = (Utf8) value;
+          return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
+        }
+        return StringData.fromString(value.toString());
+      case FIXED: // byte[]
+        if (value instanceof byte[]) {
+          return value;
+        } else if (value instanceof GenericData.Fixed) {

Review comment:
       Same question here,  would it be possible that the value is a `GenericData.Fixed` or `ByteBuffer` ?  At least the PartitionData will tranform the `ByteBuffer` to `byte[]` ? 
   https://github.com/apache/iceberg/blob/c28d1c8ba6322af07f0206c4cf3fdad875f37ac1/core/src/main/java/org/apache/iceberg/PartitionData.java#L148




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488365276



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {
+          Utf8 utf8 = (Utf8) value;
+          return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
+        }
+        return StringData.fromString(value.toString());
+      case FIXED: // byte[]
+        if (value instanceof byte[]) {
+          return value;
+        } else if (value instanceof GenericData.Fixed) {

Review comment:
       The `PartitionData` will convert `byte[]` back to `ByteBuffer`..
   https://github.com/apache/iceberg/blob/c28d1c8ba6322af07f0206c4cf3fdad875f37ac1/core/src/main/java/org/apache/iceberg/PartitionData.java#L131
   I think it is good to keep it safe.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488370446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {

Review comment:
       SQL use ordered `select`, because SQL not supports nested fields push down now.
   But I think should support nested push down in future, then, should use `project`.
   
   But if we provide an original schema ordered `select(Set<String>)` like `TableScan.select`, I think SQL can not use this one.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r486184240



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.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.core.io.InputSplitAssigner;
+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.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;

Review comment:
       nit: could we align the assignment order with the arguments order ? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477000361



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanOptions.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.Serializable;
+import java.util.Map;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+public class ScanOptions implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  public static final ConfigOption<Long> SNAPSHOT_ID =
+      ConfigOptions.key("snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Boolean> CASE_SENSITIVE =
+      ConfigOptions.key("case-sensitive").booleanType().defaultValue(false);
+
+  public static final ConfigOption<Long> AS_OF_TIMESTAMP =
+      ConfigOptions.key("as-of-timestamp").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> START_SNAPSHOT_ID =
+      ConfigOptions.key("start-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> END_SNAPSHOT_ID =
+      ConfigOptions.key("end-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_SIZE =
+      ConfigOptions.key("split-size").longType().defaultValue(null);
+
+  public static final ConfigOption<Integer> SPLIT_LOOKBACK =
+      ConfigOptions.key("split-lookback").intType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_FILE_OPEN_COST =
+      ConfigOptions.key("split-file-open-cost").longType().defaultValue(null);
+
+  private final boolean caseSensitive;
+  private final Long snapshotId;
+  private final Long startSnapshotId;
+  private final Long endSnapshotId;
+  private final Long asOfTimestamp;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final String nameMapping;
+
+  public ScanOptions(boolean caseSensitive, Long snapshotId, Long startSnapshotId, Long endSnapshotId,
+                     Long asOfTimestamp, Long splitSize, Integer splitLookback, Long splitOpenFileCost,
+                     String nameMapping) {
+    this.caseSensitive = caseSensitive;
+    this.snapshotId = snapshotId;
+    this.startSnapshotId = startSnapshotId;
+    this.endSnapshotId = endSnapshotId;
+    this.asOfTimestamp = asOfTimestamp;
+    this.splitSize = splitSize;
+    this.splitLookback = splitLookback;
+    this.splitOpenFileCost = splitOpenFileCost;
+    this.nameMapping = nameMapping;
+  }
+
+  public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
+  public Long getSnapshotId() {
+    return snapshotId;
+  }
+
+  public Long getStartSnapshotId() {
+    return startSnapshotId;
+  }
+
+  public Long getEndSnapshotId() {
+    return endSnapshotId;
+  }
+
+  public Long getAsOfTimestamp() {
+    return asOfTimestamp;
+  }
+
+  public Long getSplitSize() {
+    return splitSize;
+  }
+
+  public Integer getSplitLookback() {
+    return splitLookback;
+  }
+
+  public Long getSplitOpenFileCost() {
+    return splitOpenFileCost;
+  }
+
+  public String getNameMapping() {
+    return nameMapping;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static ScanOptions of(Map<String, String> options) {
+    return builder().options(options).build();
+  }
+
+  public static final class Builder {
+    private boolean caseSensitive = CASE_SENSITIVE.defaultValue();
+    private Long snapshotId = SNAPSHOT_ID.defaultValue();
+    private Long startSnapshotId = START_SNAPSHOT_ID.defaultValue();
+    private Long endSnapshotId = END_SNAPSHOT_ID.defaultValue();
+    private Long asOfTimestamp = AS_OF_TIMESTAMP.defaultValue();
+    private Long splitSize = SPLIT_SIZE.defaultValue();
+    private Integer splitLookback = SPLIT_LOOKBACK.defaultValue();
+    private Long splitOpenFileCost = SPLIT_FILE_OPEN_COST.defaultValue();
+    private String nameMapping;
+
+    private Builder() {
+    }
+
+    public Builder options(Map<String, String> options) {
+      Configuration config = new Configuration();
+      options.forEach(config::setString);
+      this.caseSensitive = config.get(CASE_SENSITIVE);
+      this.snapshotId = config.get(SNAPSHOT_ID);
+      this.asOfTimestamp = config.get(AS_OF_TIMESTAMP);
+      this.startSnapshotId = config.get(START_SNAPSHOT_ID);
+      this.endSnapshotId = config.get(END_SNAPSHOT_ID);
+      this.splitSize = config.get(SPLIT_SIZE);
+      this.splitLookback = config.get(SPLIT_LOOKBACK);
+      this.splitOpenFileCost = config.get(SPLIT_FILE_OPEN_COST);
+      this.nameMapping = options.get(DEFAULT_NAME_MAPPING);
+      return this;
+    }
+
+    public Builder caseSensitive(boolean newCaseSensitive) {
+      this.caseSensitive = newCaseSensitive;
+      return this;
+    }
+
+    public Builder snapshotId(Long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder startSnapshotId(Long newStartSnapshotId) {
+      this.startSnapshotId = newStartSnapshotId;
+      return this;
+    }
+
+    public Builder endSnapshotId(Long newEndSnapshotId) {
+      this.endSnapshotId = newEndSnapshotId;
+      return this;
+    }
+
+    public Builder asOfTimestamp(Long newAsOfTimestamp) {
+      this.asOfTimestamp = newAsOfTimestamp;
+      return this;
+    }
+
+    public Builder splitSize(Long newSplitSize) {
+      this.splitSize = newSplitSize;
+      return this;
+    }
+
+    public Builder splitLookback(Integer newSplitLookback) {
+      this.splitLookback = newSplitLookback;
+      return this;
+    }
+
+    public Builder splitOpenFileCost(Long newSplitOpenFileCost) {
+      this.splitOpenFileCost = newSplitOpenFileCost;
+      return this;
+    }
+
+    public Builder nameMapping(String newNameMapping) {
+      this.nameMapping = newNameMapping;
+      return this;
+    }
+
+    public ScanOptions build() {

Review comment:
       Q:  do you think whether there's need to abstract the common options builder sharing between flink and spark (maybe also hive/pig)  to validate and build those properties into a `ScanOptions` ?  If sure,  we may finish that in a new separate pr. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r473582391



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       It doesn't matter if we change the order of the schema, as long as the ID doesn't change.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476981455



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       For a stream computing job, I think it's better to have it. If there is a Job running in the cluster, in the future, if user update Iceberg-Flink version, this version modify something that does not affect compatibility, but resulting in a change to `serialVersionUID`, the user's job will be incompatible after the cluster upgrade. In fact, this situation is compatible.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476977632



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -127,6 +128,11 @@ public Integer nonNullRead(ColumnVector vector, int row) {
     @Override
     public DecimalData nonNullRead(ColumnVector vector, int row) {
       HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row];
+
+      // The hive ORC writer may will adjust the scale of decimal data.
+      Preconditions.checkArgument(value.precision() <= precision,
+          "Cannot read value as decimal(%s,%s), too large: %s", precision, scale, value);

Review comment:
       I think it is better to add this check to avoid potential precision mismatched bugs.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472043361



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       Not the cases you said,  I read the cases in `TestReadProject`, and was thought that we may also need a ut to address this one:  write records into a table and then read them from table by another schema with same fields id but different field names. But we flink are prejection with a `List<String>` (field names),  so we don't have such case.   We could ignore the case for unit test.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477177976



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }

Review comment:
       Do we really need those three methods ?  I saw that we would `loadTable` and override all of the three if anyone is null, that says setting one or two of them won't work in this builder. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487612432



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanOptions.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.Serializable;
+import java.util.Map;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+public class ScanOptions implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  public static final ConfigOption<Long> SNAPSHOT_ID =
+      ConfigOptions.key("snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Boolean> CASE_SENSITIVE =
+      ConfigOptions.key("case-sensitive").booleanType().defaultValue(false);
+
+  public static final ConfigOption<Long> AS_OF_TIMESTAMP =
+      ConfigOptions.key("as-of-timestamp").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> START_SNAPSHOT_ID =
+      ConfigOptions.key("start-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> END_SNAPSHOT_ID =
+      ConfigOptions.key("end-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_SIZE =
+      ConfigOptions.key("split-size").longType().defaultValue(null);
+
+  public static final ConfigOption<Integer> SPLIT_LOOKBACK =
+      ConfigOptions.key("split-lookback").intType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_FILE_OPEN_COST =
+      ConfigOptions.key("split-file-open-cost").longType().defaultValue(null);
+
+  private final boolean caseSensitive;
+  private final Long snapshotId;
+  private final Long startSnapshotId;
+  private final Long endSnapshotId;
+  private final Long asOfTimestamp;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final String nameMapping;
+
+  public ScanOptions(boolean caseSensitive, Long snapshotId, Long startSnapshotId, Long endSnapshotId,
+                     Long asOfTimestamp, Long splitSize, Integer splitLookback, Long splitOpenFileCost,
+                     String nameMapping) {
+    this.caseSensitive = caseSensitive;
+    this.snapshotId = snapshotId;
+    this.startSnapshotId = startSnapshotId;
+    this.endSnapshotId = endSnapshotId;
+    this.asOfTimestamp = asOfTimestamp;
+    this.splitSize = splitSize;
+    this.splitLookback = splitLookback;
+    this.splitOpenFileCost = splitOpenFileCost;
+    this.nameMapping = nameMapping;
+  }
+
+  public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
+  public Long getSnapshotId() {

Review comment:
       I'm not sure if this class should conform to the typical style used by Flink or Iceberg, but in Iceberg, we omit `get` from getter names because it doesn't add any helpful context and is awkward in non-Java languages where getter methods are named for fields.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r507546843



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}.
+   * See more options in {@link ScanOptions}.
+   * <p>
+   * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and
+   * read records incrementally.
+   * The Bounded and Unbounded depends on the {@link Builder#options(ScanOptions)}:
+   * <ul>
+   *   <li>Without startSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId and with endSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded</li>
+   * </ul>
+   * <p>
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData() {

Review comment:
       I think it is better to keep builder pattern. And for now, we can create a `FlinkInputFormat` without env too.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r486189896



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {

Review comment:
       nit: how about importing this `Configuration` explicitly in the import part ?  I did not see the duplicated `Configuration` classes are used in this file.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475309860



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -246,7 +256,7 @@ public void setBatchContext(long batchOffsetInFile) {
 
     StructReader(List<OrcValueReader<?>> readers, Types.StructType struct, Map<Integer, ?> idToConstant) {
       super(readers, struct, idToConstant);
-      this.numFields = readers.size();
+      this.numFields = struct.fields().size();

Review comment:
       This fixes Flink Orc Reader (with partition) bug.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476930870



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -127,6 +128,11 @@ public Integer nonNullRead(ColumnVector vector, int row) {
     @Override
     public DecimalData nonNullRead(ColumnVector vector, int row) {
       HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row];
+
+      // The hive ORC writer may will adjust the scale of decimal data.
+      Preconditions.checkArgument(value.precision() <= precision,
+          "Cannot read value as decimal(%s,%s), too large: %s", precision, scale, value);

Review comment:
       Does Flink require this?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476933977



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {
+          Utf8 utf8 = (Utf8) value;
+          return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
+        }
+        return StringData.fromString(value.toString());
+      case FIXED: // byte[]
+        if (value instanceof byte[]) {
+          return value;
+        } else if (value instanceof GenericData.Fixed) {
+          return ((GenericData.Fixed) value).bytes();
+        }
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case BINARY: // byte[]
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case TIME: // int instead of long

Review comment:
       Nit: int in millis?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-698682917


   Thanks @rdblue for the review, I'll address your test comments in next PR (Integrate to SQL).
   The reason for a separate `ScanOptions` is: For SQL layer, `ScanOptions` contains options in `CREATE TABLE ... WITH (options)`, while other parameters of `FlinkSource.Builder` are not. This means that the parameters in `ScanOptions` must be in the form of string.
   If you don't think it is necessary, we can also merge it into the builder.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487604622



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {

Review comment:
       We don't normally include the type returned by the builder in the build method unless it is distinguishing between two options (like `build` and `buildUnchecked`). Could this just be `build`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475317848



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java
##########
@@ -246,7 +256,7 @@ public void setBatchContext(long batchOffsetInFile) {
 
     StructReader(List<OrcValueReader<?>> readers, Types.StructType struct, Map<Integer, ?> idToConstant) {
       super(readers, struct, idToConstant);
-      this.numFields = readers.size();
+      this.numFields = struct.fields().size();

Review comment:
       Nice catch,  if the schema is a projected read schema,  then the `numFields` will be mismatched.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487602928



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       Could it also set `tasks` to `null`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472026626



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       > I'd prefer to use the correct projected schema to read the target RowData if possible
   
   This is what I want too, but I'm afraid the current format readers do not have this capability. You can take a look to `AvroSchemaWithTypeVisitor`, the readers order is according to file schema instead of Flink projected/expected schema.
   
   > Because this is in the critical read path and an extra RowData transformation will cost more resources
   
   The performance is OK, because we just use a lazy projection in `ProjectionRowData`, Unnecessary projections are omitted.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476981455



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       For a stream computing job, I think it's better to have it. Because in the future, if we modify something that does not affect compatibility, resulting in a change to `serialVersionUID`, the user's job will be incompatible after the cluster upgrade. In fact, this situation is compatible.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r493999772



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       OH, Sorry, I missed the comment, I am OK to set tasks to null, just this removes the final attribute of `tasks`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476934618



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       Is this needed?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477164853



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public FlinkInputFormat build() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null.");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      // load required fields by table loader.
+      if (icebergSchema == null || io == null || encryption == null) {
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          Table table = loader.loadTable();
+          this.icebergSchema = table.schema();
+          this.io = table.io();
+          this.encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project.");
+      }
+
+      TableSchema flinkProjectedSchema = projectedSchema;
+
+      if (selectedFields != null) {
+        TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(() -> new IllegalArgumentException(
+              "The fields are illegal in projectedFields: " + selectedFields));
+          builder.field(column.getName(), column.getType());
+          flinkProjectedSchema = builder.build();

Review comment:
       Yes




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475302981



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {

Review comment:
       `projectFields.length ==0` means project empty column. But `projectedFields == null` means project all 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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477003569



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testProjection() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> inputRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords);
+    assertRows(execute(table, Collections.singletonList("data")), Row.of(inputRecords.get(0).get(0)));
+  }
+
+  @Test
+  public void testIdentityPartitionProjections() throws Exception {
+    Schema logSchema = new Schema(

Review comment:
       Q: Do we need more cases to address the other partitioned data type in https://github.com/apache/iceberg/pull/1346/files#diff-84728688cba8556f9ff91f32d3873efcR112 ? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r473480966



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       You should be able to use the expected/projection schema. All readers should reorder columns to produce the requested column order.
   
   `AvroSchemaWithTypeVisitor` traverses the file schema to create the reader structure, but that's because fields in Avro must be read in the file's order. But when that reader adds data columns to records, the values are put in the correct order because the [`ResolvingDecoder` returns the correct position](https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java#L645-L648) in the projection schema.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480635272



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.FixupTypes;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+/**
+ * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one,
+ * which may not be correct.
+ */
+class FlinkFixupTypes extends FixupTypes {
+
+  private FlinkFixupTypes(Schema referenceSchema) {
+    super(referenceSchema);
+  }
+
+  static Schema fixup(Schema schema, Schema referenceSchema) {
+    return new Schema(TypeUtil.visit(schema,
+        new FlinkFixupTypes(referenceSchema)).asStructType().fields());
+  }
+
+  @Override
+  protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) {
+    if (type instanceof Types.FixedType) {
+      int length = ((Types.FixedType) type).length();
+      return source.typeId() == Type.TypeID.UUID && length == 16;

Review comment:
       Yes




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r474609045



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/RowDataIterator.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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 org.apache.flink.table.data.RowData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+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.flink.data.FlinkParquetReaders;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.mapping.NameMappingParser;
+import org.apache.iceberg.parquet.Parquet;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PartitionUtil;
+
+class RowDataIterator extends DataIterator<RowData> {
+
+  private final Schema projectedSchema;
+  private final String nameMapping;
+  private final boolean caseSensitive;
+
+  RowDataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption, Schema projectedSchema,
+                  String nameMapping, boolean caseSensitive) {
+    super(task, fileIo, encryption);
+    this.projectedSchema = projectedSchema;
+    this.nameMapping = nameMapping;
+    this.caseSensitive = caseSensitive;
+  }
+
+  @Override
+  protected CloseableIterator<RowData> openTaskIterator(FileScanTask task) {
+    Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds());
+
+    Map<Integer, ?> idToConstant = partitionSchema.columns().isEmpty() ? ImmutableMap.of() :
+        PartitionUtil.constantsMap(task, RowDataIterator::convertConstant);
+    CloseableIterable<RowData> iterable = newIterable(task, idToConstant);
+    return iterable.iterator();
+  }
+
+  private CloseableIterable<RowData> newIterable(FileScanTask task, Map<Integer, ?> idToConstant) {
+    CloseableIterable<RowData> iter;
+    if (task.isDataTask()) {
+      throw new UnsupportedOperationException("Cannot read data task.");
+    } else {
+      switch (task.file().format()) {
+        case PARQUET:
+          iter = newParquetIterable(task, idToConstant);
+          break;
+
+        case AVRO:
+          iter = newAvroIterable(task, idToConstant);
+          break;
+

Review comment:
       Now the orc reader has been merge into master,  pls add the orc iterable here . 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-680698918


   Thanks @rdblue and @openinx for the review, I have added `project(TableSchema schema)` and `select(List<String> fields)` to format builder, `project` can support nested projection.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488403616



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       I was thinking about adding a new config option like `bounded` is true or not.
   After reading your comments, I think it's very good. Because streaming jobs have a high probability of setting the starting snapshot ID or timestamp, the strategy can be:
   - without startSnapshotId: bounded
   - with startSnapshotId and endSnapshotId: bounded
   - with startSnapshotId and without endSnapshotId: unbounded
   - with startSnapshotId -1 (means unbounded preceding) and without endSnapshotId: unbounded
   
   In this way, we can have a unify builder.
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488365948



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {

Review comment:
       I think it is good to keep it safe.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480593095



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java
##########
@@ -0,0 +1,51 @@
+/*
+ * 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;
+
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.FixupTypes;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.types.Types;
+
+/**
+ * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one,
+ * which may not be correct.
+ */
+class FlinkFixupTypes extends FixupTypes {
+
+  private FlinkFixupTypes(Schema referenceSchema) {
+    super(referenceSchema);
+  }
+
+  static Schema fixup(Schema schema, Schema referenceSchema) {
+    return new Schema(TypeUtil.visit(schema,
+        new FlinkFixupTypes(referenceSchema)).asStructType().fields());
+  }
+
+  @Override
+  protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) {
+    if (type instanceof Types.FixedType) {
+      int length = ((Types.FixedType) type).length();
+      return source.typeId() == Type.TypeID.UUID && length == 16;

Review comment:
       We have a discussion in https://github.com/apache/iceberg/issues/1302 for removing UUID. Is this a temporary solution?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480656544



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       Does `ConbinedScanTask`  happen to have many tasks? If not, this is ok.  Otherwise, maybe we could use a flag and check it in `updateCurrentIterator` to avoid loop.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r474593402



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *

Review comment:
       we may need to add a comment to indicate that:  we don't support complex data type projection for flink now. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {

Review comment:
       nit : `projectedFields == null || projectFields.length ==0`

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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;

Review comment:
       Here, you put all reader related classes inside the `source` package,  will we also need to put those writer related classes into `sink` package ?    I don't have strong feeling to do that, keeping consistence is OK for me. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       We've already have a `Schema#select`, will it fit for your requirement ? 
   
   ```
     /**
      * Creates a projection schema for a subset of columns, selected by name.
      * <p>
      * Names that identify nested fields will select part or all of the field's top-level column.
      *
      * @param names a List of String names for selected columns
      * @return a projection schema from this schema, by name
      */
     public Schema select(Collection<String> names) {
       return internalSelect(names, true);
     }
   ```




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494690367



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;

Review comment:
       I find it really strange that this is delegated to a subclass, given that it builds a very specific nested projection.
   
   Why not make this use a method like `execute(Table, List<String>)`, but pass in the projection instead of a list of fields?
   
   Then you could keep all of the schema details in the test method here, rather than delegating this assertion. I think it would be cleaner.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494662994



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       Field reordering tests are at the file format level. Each file format has to be able to project columns in the requested order. So any reordered schema should work as long as it is passed down correctly, which is what the new `convert` method does.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r474605890



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,218 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {

Review comment:
       Em, seems I've missed to close the `TableLoader`  in `IcebergFilesCommitter` patch..




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476927021



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *

Review comment:
       Why aren't nested fields supported?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476981455



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       For a stream computing task, I think it's better to have it. Because in the future, if we modify something that does not affect compatibility, resulting in a change to `serialVersionUID`, the user's job will be incompatible after the cluster upgrade. In fact, this situation is compatible.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476930537



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java
##########
@@ -39,18 +39,14 @@
 public class FlinkOrcReader implements OrcRowReader<RowData> {
   private final OrcValueReader<?> reader;
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
     this(iSchema, readSchema, ImmutableMap.of());
   }
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
     this.reader = OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant));
   }
 
-  public static OrcRowReader<RowData> buildReader(Schema schema, TypeDescription readSchema) {

Review comment:
       Was this not used anywhere?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488379347



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();

Review comment:
       I will remove this `select`, because what Flink SQL want, is an order changed select, instead of using original iceberg table order.
   We should provide a unified `select`, so I think we can provide in Flink side now, and we can use `project`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487604832



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {

Review comment:
       Is this needed? If this delegated to when the scan is built, then the scan would do the check and users would get consistent exception messages.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494735542



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;

Review comment:
       I think we can move `testNestedProjection` to `TestFlinkInputFormat`, because only InputFormat supports nested push-down, SQL can not.

##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;

Review comment:
       This is also because of the difference between InputFormat and SQL(It can actually filter out the data). I think I should put it into a subclass of SQL.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Jiayi-Liao commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
Jiayi-Liao commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r507540278



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}.
+   * See more options in {@link ScanOptions}.
+   * <p>
+   * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and
+   * read records incrementally.
+   * The Bounded and Unbounded depends on the {@link Builder#options(ScanOptions)}:
+   * <ul>
+   *   <li>Without startSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId and with endSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded</li>
+   * </ul>
+   * <p>
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData() {

Review comment:
       Hi @JingsongLi , I'm testing Iceberg recently. Since the StreamExecutionEnvironment is a must-have parameter for FlinkSource, would it better to put it in the builder's constructor instead of FlinkSource.forRowData().env(xx)?
   
   just a minor improvement on user experience.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477178511



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }

Review comment:
       For me , the `table(Table newTable)` and `tableLoader(TableLoader newLoader)` is enough. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480637502



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       This ensures that it does not return data after it is closed. (`hasNext` return false).




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494735864



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;

Review comment:
       This is also because of the difference between InputFormat and SQL(It can actually filter out the data). I think I should put it into a subclass of SQL.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r486181821



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {

Review comment:
       Q:  is it possible that  we will step into this `if` block ?  I saw `PartitionData` will transform the `Utf8` to `String` ? 
   https://github.com/apache/iceberg/blob/c28d1c8ba6322af07f0206c4cf3fdad875f37ac1/core/src/main/java/org/apache/iceberg/PartitionData.java#L142




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r493986182



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       @JingsongLi, could you take a look at this? Do we need to exhaust the iterator, or can we simply discard the reference to it? If it needs to be closed, then we could do that and then set the reference to null.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r489125419



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       I prefer that default is bounded.
   - Bounded mode is more common in attempt and startup of users.
   - In unbounded mode, users often define `startSnapshotId `. But in bounded mode, `endSnapshotID` is rare. If default is unbounded, it is hard to define bounded mode.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] Jiayi-Liao commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
Jiayi-Liao commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r507549677



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}.
+   * See more options in {@link ScanOptions}.
+   * <p>
+   * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and
+   * read records incrementally.
+   * The Bounded and Unbounded depends on the {@link Builder#options(ScanOptions)}:
+   * <ul>
+   *   <li>Without startSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId and with endSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded</li>
+   * </ul>
+   * <p>
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData() {

Review comment:
       I see... The builder is for both InputFormat and DataStreamSource.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488367019



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.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.core.io.InputSplitAssigner;
+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.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {

Review comment:
       Yes, 
   - Planner: `createInputSplits` and `getInputSplitAssigner` will be called in the JobManager
   - Reader: `open`, `reachedEnd`, `nextRecord` and `close` will be called in the TaskManager




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488370446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {

Review comment:
       SQL use order changed `select`, because SQL not supports nested fields push down now.
   But I think should support nested push down in future, then, should use `project`.
   
   But if we provide an original schema ordered `select(Set<String>)` like `TableScan.select`, I think SQL can not use this one.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r486188595



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.

Review comment:
       We could also use this source to read multiple snapshots in bounded mode , right ?  Since it's a public interface/method exposed to user, I'd prefer to provide a more detailed javadoc.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477076138



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       Create https://github.com/apache/iceberg/pull/1382 for fixup.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477055784



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,378 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testProjection() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> inputRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords);
+    assertRows(execute(table, Collections.singletonList("data")), Row.of(inputRecords.get(0).get(0)));
+  }
+
+  @Test
+  public void testIdentityPartitionProjections() throws Exception {
+    Schema logSchema = new Schema(

Review comment:
       There is `testPartitionTypes` for this.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488370446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {

Review comment:
       SQL use ordered `select`, because SQL not supports nested fields push down now.
   But I think should support nested push down in future, then, should use `project`.
   
   But if we provide a unordered `select(Set<String>)` like `TableScan.select`, I think SQL can not use this one.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476929916



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       We do this in Spark here: https://github.com/apache/iceberg/blob/master/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java#L161-L166
   
   You might try a similar approach since we already have Flink to Iceberg conversion. It should just be a matter of reassigning the IDs. You may also need the type fixes, I don't recall if Iceberg to Flink to Iceberg conversion is lossy or not.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488813468



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       Sounds like a good way to configure this to me, except that we will want to make sure the default is reasonable. For Flink, should the default be unbounded or bounded?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477146998



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public FlinkInputFormat build() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null.");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      // load required fields by table loader.
+      if (icebergSchema == null || io == null || encryption == null) {
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          Table table = loader.loadTable();
+          this.icebergSchema = table.schema();
+          this.io = table.io();
+          this.encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project.");
+      }
+
+      TableSchema flinkProjectedSchema = projectedSchema;
+
+      if (selectedFields != null) {
+        TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(() -> new IllegalArgumentException(
+              "The fields are illegal in projectedFields: " + selectedFields));
+          builder.field(column.getName(), column.getType());
+          flinkProjectedSchema = builder.build();

Review comment:
       We need to move this line out of the `for` loop ? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-678024616


   Hi @openinx and @rdblue , rebased and added case and fixed bug.
   I think this is ready to review.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488370446



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {

Review comment:
       use `select`, because SQL not supports nested fields push down now.
   But I think should support nested push down in future, then, should use `project`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477191361



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,255 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }

Review comment:
       OK I can remove them.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480675134



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
##########
@@ -0,0 +1,115 @@
+/*
+ * 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.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+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.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Schema;
+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 + "/default/t"));
+  }
+
+  @Override
+  protected List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId, Long startSnapshotId,
+      Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter) throws IOException {
+    ScanOptions options = ScanOptions.builder().snapshotId(snapshotId).startSnapshotId(startSnapshotId)
+        .endSnapshotId(endSnapshotId).asOfTimestamp(asOfTimestamp).build();
+    if (loader != null) {
+      builder.tableLoader(TableLoader.fromCatalog(loader, TableIdentifier.of("default", "t")));
+    }
+
+    return run(builder.select(projectFields).filters(filters).options(options).build());
+  }
+
+  @Override
+  protected void assertResiduals(
+      Schema shcema, List<Row> results, List<Record> writeRecords, List<Record> filteredRecords) {
+    // can not filter the data.
+    assertRecords(results, writeRecords, shcema);

Review comment:
       nit: typo shcema ->schema.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477147843



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  private FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {
+    this.iterator = new RowDataIterator(split.getTask(), io, encryption, projectedSchema,
+                                        options.getNameMapping(), options.isCaseSensitive());
+  }
+
+  @Override
+  public boolean reachedEnd() {
+    return !iterator.hasNext();
+  }
+
+  @Override
+  public RowData nextRecord(RowData reuse) {
+    return iterator.next();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (iterator != null) {
+      iterator.close();
+    }
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static final class Builder {
+    private TableLoader tableLoader;
+    private Schema icebergSchema;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private FileIO io;
+    private EncryptionManager encryption;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private Builder() {
+    }
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.icebergSchema = newTable.schema();
+      this.io = newTable.io();
+      this.encryption = newTable.encryption();
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder icebergSchema(Schema newSchema) {
+      this.icebergSchema = newSchema;
+      return this;
+    }
+
+    public Builder io(FileIO newIO) {
+      this.io = newIO;
+      return this;
+    }
+
+    public Builder encryption(EncryptionManager newEncryption) {
+      this.encryption = newEncryption;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public FlinkInputFormat build() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null.");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      // load required fields by table loader.
+      if (icebergSchema == null || io == null || encryption == null) {
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          Table table = loader.loadTable();
+          this.icebergSchema = table.schema();
+          this.io = table.io();
+          this.encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project.");
+      }
+
+      TableSchema flinkProjectedSchema = projectedSchema;
+
+      if (selectedFields != null) {
+        TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(() -> new IllegalArgumentException(
+              "The fields are illegal in projectedFields: " + selectedFields));

Review comment:
       I think we need to point out which column is missing in the error message .




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487610671



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {

Review comment:
       Will SQL use this or `select`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488379724



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {

Review comment:
       See below comments.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494688130



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords, SCHEMA);
+  }
+
+  @Test
+  public void testProjection() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> inputRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords);
+    assertRows(execute(table, Collections.singletonList("data")), Row.of(inputRecords.get(0).get(0)));
+  }
+
+  @Test
+  public void testIdentityPartitionProjections() throws Exception {
+    Schema logSchema = new Schema(
+        Types.NestedField.optional(1, "id", Types.IntegerType.get()),
+        Types.NestedField.optional(2, "dt", Types.StringType.get()),
+        Types.NestedField.optional(3, "level", Types.StringType.get()),
+        Types.NestedField.optional(4, "message", Types.StringType.get())
+    );
+    PartitionSpec spec =
+        PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build();
+
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), logSchema, spec);
+    List<Record> inputRecords = RandomGenericData.generate(logSchema, 10, 0L);
+
+    int idx = 0;
+    AppendFiles append = table.newAppend();
+    for (Record record : inputRecords) {
+      record.set(1, "2020-03-2" + idx);
+      record.set(2, Integer.toString(idx));
+      append.appendFile(new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).writeFile(
+          org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), ImmutableList.of(record)));
+      idx += 1;
+    }
+    append.commit();
+
+    // individual fields
+    validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords);
+    validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords);
+    // field pairs
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords);
+    // out-of-order pairs
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords);
+    // out-of-order triplets
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "level", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "dt", "message"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("dt", "message", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("level", "message", "dt"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "dt", "level"), inputRecords);
+    validateIdentityPartitionProjections(table, Arrays.asList("message", "level", "dt"), inputRecords);
+  }
+
+  private void validateIdentityPartitionProjections(Table table, List<String> projectedFields,
+      List<Record> inputRecords) throws IOException {
+    List<Row> rows = execute(table, projectedFields);
+
+    for (int pos = 0; pos < inputRecords.size(); pos++) {
+      Record inputRecord = inputRecords.get(pos);
+      Row actualRecord = rows.get(pos);
+
+      for (int i = 0; i < projectedFields.size(); i++) {
+        String name = projectedFields.get(i);
+        Assert.assertEquals(
+            "Projected field " + name + " should match", inputRecord.getField(name), actualRecord.getField(i));
+      }
+    }
+  }
+
+  @Test
+  public void testSnapshotReads() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+
+    GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER);
+
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    helper.appendToTable(expectedRecords);
+    long snapshotId = table.currentSnapshot().snapshotId();
+
+    long timestampMillis = table.currentSnapshot().timestampMillis();
+
+    // produce another timestamp
+    Thread.sleep(10);

Review comment:
       We usually introduce a spin to avoid sleeping for long durations in lots of tests, like this: https://github.com/apache/iceberg/blob/master/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java#L78




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-698666137


   I'm going ahead and merging this because I think the remaining things are minor, but we will probably want to fix them.
   
   First, breaking the tests across two classes could be cleaner. In the future, I'd recommend writing tests and not breaking across a parent/child class until you want to add the other set of tests. That way it is easier to see what is needed and will be clean.
   
   Second, I don't understand the benefit of having a separate `ScanOptions` class, besides that it can pull options out of a map. It seems to me that it would be simpler to just have the `FlinkSource.Builder` class and move all of the scan options to that class.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476999912



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       > You may also need the type fixes
   
   Yes, we can have.
   
   After thinking about it with nested projection, we can pass a `TableSchema requestedSchema` parameter, which contains the required projection (Including nested) and the type to be fixed.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476979018



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,219 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+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.core.io.InputSplitAssigner;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;

Review comment:
       I just notice we don't need `serialVersionUID` in Iceberg, I think I can remove it.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487603116



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;

Review comment:
       Nit: we mostly use `io` for `FileIO` in other places.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487604022



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java
##########
@@ -0,0 +1,122 @@
+/*
+ * 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.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.core.io.InputSplitAssigner;
+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.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+
+/**
+ * Flink {@link InputFormat} for Iceberg.
+ */
+public class FlinkInputFormat extends RichInputFormat<RowData, FlinkInputSplit> {
+
+  private static final long serialVersionUID = 1L;
+
+  private final TableLoader tableLoader;
+  private final Schema projectedSchema;
+  private final ScanOptions options;
+  private final List<Expression> filterExpressions;
+  private final FileIO io;
+  private final EncryptionManager encryption;
+  private final SerializableConfiguration serializableConf;
+
+  private transient RowDataIterator iterator;
+
+  FlinkInputFormat(
+      TableLoader tableLoader, Schema projectedSchema, FileIO io, EncryptionManager encryption,
+      List<Expression> filterExpressions, ScanOptions options, SerializableConfiguration serializableConf) {
+    this.tableLoader = tableLoader;
+    this.projectedSchema = projectedSchema;
+    this.options = options;
+    this.filterExpressions = filterExpressions;
+    this.io = io;
+    this.encryption = encryption;
+    this.serializableConf = serializableConf;
+  }
+
+  @VisibleForTesting
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  @Override
+  public BaseStatistics getStatistics(BaseStatistics cachedStatistics) {
+    // Legacy method, not be used.
+    return null;
+  }
+
+  @Override
+  public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException {
+    // Called in Job manager, so it is OK to load table from catalog.
+    tableLoader.open(serializableConf.get());
+    try (TableLoader loader = tableLoader) {
+      Table table = loader.loadTable();
+      FlinkSplitGenerator generator = new FlinkSplitGenerator(table, projectedSchema, 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) {

Review comment:
       So this class functions as both planner and reader?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476983862



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java
##########
@@ -39,18 +39,14 @@
 public class FlinkOrcReader implements OrcRowReader<RowData> {
   private final OrcValueReader<?> reader;
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
     this(iSchema, readSchema, ImmutableMap.of());
   }
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
     this.reader = OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant));
   }
 
-  public static OrcRowReader<RowData> buildReader(Schema schema, TypeDescription readSchema) {

Review comment:
       It is just used by testing code.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r507546843



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableScan;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.flink.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link TableScan}.
+   * See more options in {@link ScanOptions}.
+   * <p>
+   * The Source can be read static data in bounded mode. It can also continuously check the arrival of new data and
+   * read records incrementally.
+   * The Bounded and Unbounded depends on the {@link Builder#options(ScanOptions)}:
+   * <ul>
+   *   <li>Without startSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId and with endSnapshotId: Bounded</li>
+   *   <li>With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded</li>
+   * </ul>
+   * <p>
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forRowData() {

Review comment:
       I think it is better to keep builder pattern. (For now, we can create a `FlinkInputFormat` without env too.)




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] chenjunjiedada commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r480596868



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.tasks = task.files().iterator();
+    this.fileIo = fileIo;
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator

Review comment:
       Why we need to exhaust tasks?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472019911



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       There is a `NestedFieldsProjectableTableSource` in Flink, but so far, no connector has implemented it. There may be unknown risks, and I tend not to implement it first.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r472186456



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +102,22 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Prune columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   * @throws IllegalArgumentException if the Flink type does not match the Schema
+   */
+  public static Schema pruneWithoutReordering(Schema schema, List<String> projectedFields) {
+    if (projectedFields == null) {
+      return schema;
+    }
+
+    Map<String, Integer> indexByName = TypeUtil.indexByName(schema.asStruct());
+    Set<Integer> projectedIds = projectedFields.stream().map(indexByName::get).collect(Collectors.toSet());
+    return TypeUtil.select(schema, projectedIds);

Review comment:
       cc @rdblue , Mind to take a look this ? Thanks.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475311661



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       OK, that sounds reasonable. 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346


   


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487604305



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java
##########
@@ -0,0 +1,64 @@
+/*
+ * 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 org.apache.flink.core.io.InputSplit;
+import org.apache.flink.core.io.LocatableInputSplit;
+import org.apache.iceberg.CombinedScanTask;
+
+/**
+ * TODO Implement {@link LocatableInputSplit}.
+ */
+public class FlinkInputSplit implements InputSplit {
+
+  private final int splitNumber;
+  private final CombinedScanTask task;
+
+  FlinkInputSplit(int splitNumber, CombinedScanTask task) {
+    this.splitNumber = splitNumber;
+    this.task = task;
+  }
+
+  @Override
+  public int getSplitNumber() {
+    return splitNumber;
+  }
+
+  CombinedScanTask getTask() {
+    return task;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+    FlinkInputSplit that = (FlinkInputSplit) o;
+    return splitNumber == that.splitNumber;

Review comment:
       It doesn't seem correct to ignore `task` in `equals`. This could lead to bugs in testing. For example, if someone uses this to assert that planned splits are equal to expected splits. Is this required by Flink?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487611170



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {

Review comment:
       Since these are intended to be called from child classes, should they be `protected`?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r494735542



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,385 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+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.utils.TypeConversions;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.RowDataConverter;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  private HadoopCatalog catalog;
+  protected String warehouse;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters(name = "format={0}")
+  public static Object[] parameters() {
+    return new Object[] {"avro", "parquet", "orc"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    // before variables
+    Configuration conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return execute(table, ScanOptions.builder().build());
+  }
+
+  protected abstract List<Row> execute(Table table, List<String> projectFields) throws IOException;
+
+  protected abstract List<Row> execute(Table table, ScanOptions options) throws IOException;
+
+  protected abstract List<Row> execute(Table table, List<Expression> filters, String sqlFilter) throws IOException;
+
+  /**
+   * The Flink SQL has no residuals, because there will be operator to filter all the data that should be filtered.
+   * But the FlinkInputFormat can't.
+   */
+  protected abstract void assertResiduals(Schema schema, List<Row> results, List<Record> writeRecords,
+                                          List<Record> filteredRecords) throws IOException;
+
+  /**
+   * Schema: [data, nested[f1, f2, f3], id]
+   * Projection: [nested.f2, data]
+   * The Flink SQL output: [f2, data]
+   * The FlinkInputFormat output: [nested[f2], data].
+   */
+  protected abstract void assertNestedProjection(Table table, List<Record> records) throws IOException;

Review comment:
       I think we can move `testNestedProjection` to `TestFlinkInputFormat`, because only InputFormat supports nested push-down, SQL can not.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477056189



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/ScanOptions.java
##########
@@ -0,0 +1,221 @@
+/*
+ * 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.Serializable;
+import java.util.Map;
+import org.apache.flink.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+import org.apache.flink.configuration.Configuration;
+
+import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING;
+
+public class ScanOptions implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+
+  public static final ConfigOption<Long> SNAPSHOT_ID =
+      ConfigOptions.key("snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Boolean> CASE_SENSITIVE =
+      ConfigOptions.key("case-sensitive").booleanType().defaultValue(false);
+
+  public static final ConfigOption<Long> AS_OF_TIMESTAMP =
+      ConfigOptions.key("as-of-timestamp").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> START_SNAPSHOT_ID =
+      ConfigOptions.key("start-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> END_SNAPSHOT_ID =
+      ConfigOptions.key("end-snapshot-id").longType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_SIZE =
+      ConfigOptions.key("split-size").longType().defaultValue(null);
+
+  public static final ConfigOption<Integer> SPLIT_LOOKBACK =
+      ConfigOptions.key("split-lookback").intType().defaultValue(null);
+
+  public static final ConfigOption<Long> SPLIT_FILE_OPEN_COST =
+      ConfigOptions.key("split-file-open-cost").longType().defaultValue(null);
+
+  private final boolean caseSensitive;
+  private final Long snapshotId;
+  private final Long startSnapshotId;
+  private final Long endSnapshotId;
+  private final Long asOfTimestamp;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final String nameMapping;
+
+  public ScanOptions(boolean caseSensitive, Long snapshotId, Long startSnapshotId, Long endSnapshotId,
+                     Long asOfTimestamp, Long splitSize, Integer splitLookback, Long splitOpenFileCost,
+                     String nameMapping) {
+    this.caseSensitive = caseSensitive;
+    this.snapshotId = snapshotId;
+    this.startSnapshotId = startSnapshotId;
+    this.endSnapshotId = endSnapshotId;
+    this.asOfTimestamp = asOfTimestamp;
+    this.splitSize = splitSize;
+    this.splitLookback = splitLookback;
+    this.splitOpenFileCost = splitOpenFileCost;
+    this.nameMapping = nameMapping;
+  }
+
+  public boolean isCaseSensitive() {
+    return caseSensitive;
+  }
+
+  public Long getSnapshotId() {
+    return snapshotId;
+  }
+
+  public Long getStartSnapshotId() {
+    return startSnapshotId;
+  }
+
+  public Long getEndSnapshotId() {
+    return endSnapshotId;
+  }
+
+  public Long getAsOfTimestamp() {
+    return asOfTimestamp;
+  }
+
+  public Long getSplitSize() {
+    return splitSize;
+  }
+
+  public Integer getSplitLookback() {
+    return splitLookback;
+  }
+
+  public Long getSplitOpenFileCost() {
+    return splitOpenFileCost;
+  }
+
+  public String getNameMapping() {
+    return nameMapping;
+  }
+
+  public static Builder builder() {
+    return new Builder();
+  }
+
+  public static ScanOptions of(Map<String, String> options) {
+    return builder().options(options).build();
+  }
+
+  public static final class Builder {
+    private boolean caseSensitive = CASE_SENSITIVE.defaultValue();
+    private Long snapshotId = SNAPSHOT_ID.defaultValue();
+    private Long startSnapshotId = START_SNAPSHOT_ID.defaultValue();
+    private Long endSnapshotId = END_SNAPSHOT_ID.defaultValue();
+    private Long asOfTimestamp = AS_OF_TIMESTAMP.defaultValue();
+    private Long splitSize = SPLIT_SIZE.defaultValue();
+    private Integer splitLookback = SPLIT_LOOKBACK.defaultValue();
+    private Long splitOpenFileCost = SPLIT_FILE_OPEN_COST.defaultValue();
+    private String nameMapping;
+
+    private Builder() {
+    }
+
+    public Builder options(Map<String, String> options) {
+      Configuration config = new Configuration();
+      options.forEach(config::setString);
+      this.caseSensitive = config.get(CASE_SENSITIVE);
+      this.snapshotId = config.get(SNAPSHOT_ID);
+      this.asOfTimestamp = config.get(AS_OF_TIMESTAMP);
+      this.startSnapshotId = config.get(START_SNAPSHOT_ID);
+      this.endSnapshotId = config.get(END_SNAPSHOT_ID);
+      this.splitSize = config.get(SPLIT_SIZE);
+      this.splitLookback = config.get(SPLIT_LOOKBACK);
+      this.splitOpenFileCost = config.get(SPLIT_FILE_OPEN_COST);
+      this.nameMapping = options.get(DEFAULT_NAME_MAPPING);
+      return this;
+    }
+
+    public Builder caseSensitive(boolean newCaseSensitive) {
+      this.caseSensitive = newCaseSensitive;
+      return this;
+    }
+
+    public Builder snapshotId(Long newSnapshotId) {
+      this.snapshotId = newSnapshotId;
+      return this;
+    }
+
+    public Builder startSnapshotId(Long newStartSnapshotId) {
+      this.startSnapshotId = newStartSnapshotId;
+      return this;
+    }
+
+    public Builder endSnapshotId(Long newEndSnapshotId) {
+      this.endSnapshotId = newEndSnapshotId;
+      return this;
+    }
+
+    public Builder asOfTimestamp(Long newAsOfTimestamp) {
+      this.asOfTimestamp = newAsOfTimestamp;
+      return this;
+    }
+
+    public Builder splitSize(Long newSplitSize) {
+      this.splitSize = newSplitSize;
+      return this;
+    }
+
+    public Builder splitLookback(Integer newSplitLookback) {
+      this.splitLookback = newSplitLookback;
+      return this;
+    }
+
+    public Builder splitOpenFileCost(Long newSplitOpenFileCost) {
+      this.splitOpenFileCost = newSplitOpenFileCost;
+      return this;
+    }
+
+    public Builder nameMapping(String newNameMapping) {
+      this.nameMapping = newNameMapping;
+      return this;
+    }
+
+    public ScanOptions build() {

Review comment:
       I don't know, there are some difference between Flink/Spark and Hive, maybe we can try to do something in Spark.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488406696



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+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.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Schema;
+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 FlinkSource.Builder builder;
+
+  public TestFlinkInputFormat(String fileFormat) {
+    super(fileFormat);
+  }
+
+  @Override
+  public void before() throws IOException {
+    super.before();
+    builder = FlinkSource.forBounded().tableLoader(TableLoader.fromHadoopTable(warehouse + "/default/t"));
+  }
+
+  @Override
+  protected List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId, Long startSnapshotId,
+      Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter) throws IOException {
+    ScanOptions options = ScanOptions.builder().snapshotId(snapshotId).startSnapshotId(startSnapshotId)
+        .endSnapshotId(endSnapshotId).asOfTimestamp(asOfTimestamp).build();

Review comment:
       Agree with you.
   I was thinking about how to make it easy for SQL tests to reuse it. SQL testing can also rebuild SQL strings from `ScanOptions`.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r476974613



##########
File path: flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java
##########
@@ -39,18 +39,14 @@
 public class FlinkOrcReader implements OrcRowReader<RowData> {
   private final OrcValueReader<?> reader;
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) {
     this(iSchema, readSchema, ImmutableMap.of());
   }
 
-  private FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
+  public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map<Integer, ?> idToConstant) {
     this.reader = OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant));
   }
 
-  public static OrcRowReader<RowData> buildReader(Schema schema, TypeDescription readSchema) {

Review comment:
       Because there are two constructors, so we need to have two static helpers, I think we can use constructors directly.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-689253802


   > Is this one next to review?
   
   Yes~


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488379347



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();

Review comment:
       I will remove this `select`, because what Flink SQL want, is an order changed select, instead of using original iceberg table order.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475302849



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       You can take a look to `Schema.internalSelect`, it is a `Set<Integer> selected`, so actually the interface is: "selectWithoutReordering"




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-677880415


   @JingsongLi, now that #1332 has been merged, could you update this one? It looks like this one is next to review.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-699056581


   Let's take a look at `ScanOptions` in the next PR then. I would prefer to keep user-facing APIs simple, rather than leaking a SQL concern (options come from `WITH`) to users (need to use two builders). Since SQL will most likely use the `fromProperties` method, it may make sense to use a single builder, add `withProperties`, and pass properties from SQL as a map.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487605004



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();

Review comment:
       Why project the Flink schema manually rather than using `icebergSchema.select(selectedFIelds)` and converting the result?




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r474599222



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.fileIo = fileIo;

Review comment:
       nit: better to keep the assign order with the arguments ?

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,144 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption) {
+    this.fileIo = fileIo;
+    this.tasks = task.files().iterator();
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = openTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> openTaskIterator(FileScanTask scanTask) throws IOException;
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    this.currentIterator.close();
+
+    // exhaust the task iterator
+    while (tasks.hasNext()) {
+      tasks.next();
+    }
+  }
+
+  static Object convertConstant(Type type, Object value) {
+    if (value == null) {
+      return null;
+    }
+
+    switch (type.typeId()) {
+      case DECIMAL: // DecimalData
+        Types.DecimalType decimal = (Types.DecimalType) type;
+        return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale());
+      case STRING: // StringData
+        if (value instanceof Utf8) {
+          Utf8 utf8 = (Utf8) value;
+          return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength());
+        }
+        return StringData.fromString(value.toString());
+      case FIXED: // byte[]
+        if (value instanceof byte[]) {
+          return value;
+        } else if (value instanceof GenericData.Fixed) {
+          return ((GenericData.Fixed) value).bytes();
+        }
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case BINARY: // byte[]
+        return ByteBuffers.toByteArray((ByteBuffer) value);
+      case TIME: // int instead of long
+        return (int) (DateTimeUtil.timeFromMicros((Long) value).toNanoOfDay() / 1000_000);

Review comment:
       the long value is surely microseconds ?  could we just return `(int)((long)value/1_000)` ? 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488403616



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       I was thinking about adding a new config option like `bounded` is true or not.
   After reading your comments, I think it's very good. Because streaming jobs have a high probability of setting the starting snapshot ID or timestamp, the strategy can be:
   - Without startSnapshotId: Bounded
   - With startSnapshotId and with endSnapshotId: Bounded
   - With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded
   
   In this way, we can have a unify builder.
   




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r475303475



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *

Review comment:
       Add comments: Don't support nested fields projection for Flink now.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r471985446



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       I'm not quite sure whether flink support complex data type projection, if sure we may need more unit tests to address the projection cases, such as projection by a nested struct, map, list (similar to the spark's `TestReadProjection` ). 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/RowDataIterator.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.List;
+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 RowDataIterator extends DataIterator<RowData> {
+
+  private final String nameMapping;
+
+  RowDataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption, Schema tableSchema,
+                  List<String> projectedFields, String nameMapping) {
+    super(task, fileIo, encryption, tableSchema, projectedFields);
+    this.nameMapping = nameMapping;
+  }
+
+  @Override
+  protected CloseableIterator<RowData> nextTaskIterator(FileScanTask task) {
+    // schema or rows returned by readers

Review comment:
       nit: is this comment still valuable ? Seems I did not get the point. 

##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -0,0 +1,168 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.stream.Collectors;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.util.Utf8;
+import org.apache.flink.table.data.DecimalData;
+import org.apache.flink.table.data.StringData;
+import org.apache.flink.table.data.TimestampData;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.encryption.EncryptedFiles;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.util.ByteBuffers;
+import org.apache.iceberg.util.DateTimeUtil;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+abstract class DataIterator<T> implements CloseableIterator<T> {
+
+  private final Iterator<FileScanTask> tasks;
+  private final FileIO fileIo;
+  private final EncryptionManager encryption;
+  private final Schema projectedSchema;
+  private final int[] fieldsReorder;
+
+  private CloseableIterator<T> currentIterator;
+
+  DataIterator(CombinedScanTask task, FileIO fileIo, EncryptionManager encryption, Schema tableSchema,
+               List<String> projectedFields) {
+    this.fileIo = fileIo;
+    this.tasks = task.files().iterator();
+    this.encryption = encryption;
+    this.currentIterator = CloseableIterator.empty();
+
+    this.projectedSchema = FlinkSchemaUtil.pruneWithoutReordering(tableSchema, projectedFields);
+
+    // The projected schema is the schema without reordering, but Flink wants its own order, so we need to reorder the
+    // output row.
+    List<String> projectedNames = projectedSchema.asStruct().fields().stream()
+        .map(Types.NestedField::name).collect(Collectors.toList());
+    this.fieldsReorder = projectedFields == null ?
+        null : projectedFields.stream().mapToInt(projectedNames::indexOf).toArray();
+  }
+
+  Schema projectedSchema() {
+    return projectedSchema;
+  }
+
+  int[] fieldsReorder() {
+    return fieldsReorder;
+  }
+
+  InputFile getInputFile(FileScanTask task) {
+    Preconditions.checkArgument(!task.isDataTask(), "Invalid task type");
+    return encryption.decrypt(EncryptedFiles.encryptedInput(
+        this.fileIo.newInputFile(task.file().path().toString()),
+        task.file().keyMetadata()));
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    return currentIterator.next();
+  }
+
+  /**
+   * Updates the current iterator field to ensure that the current Iterator
+   * is not exhausted.
+   */
+  private void updateCurrentIterator() {
+    try {
+      while (!currentIterator.hasNext() && tasks.hasNext()) {
+        currentIterator.close();
+        currentIterator = nextTaskIterator(tasks.next());
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  abstract CloseableIterator<T> nextTaskIterator(FileScanTask scanTask) throws IOException;

Review comment:
       How about renaming this method to `openTaskIterator`  ?  

##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java
##########
@@ -0,0 +1,340 @@
+/*
+ * 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.File;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Locale;
+import org.apache.flink.test.util.AbstractTestBase;
+import org.apache.flink.types.Row;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.iceberg.AppendFiles;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.PartitionSpec;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.GenericAppenderHelper;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.flink.CatalogLoader;
+import org.apache.iceberg.hadoop.HadoopCatalog;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+@RunWith(Parameterized.class)
+public abstract class TestFlinkScan extends AbstractTestBase {
+
+  private static final Schema SCHEMA = new Schema(
+          required(1, "data", Types.StringType.get()),
+          required(2, "id", Types.LongType.get()),
+          required(3, "dt", Types.StringType.get()));
+
+  private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA)
+          .identity("dt")
+          .bucket("id", 1)
+          .build();
+
+  // before variables
+  private Configuration conf;
+  String warehouse;
+  private HadoopCatalog catalog;
+
+  // parametrized variables
+  private final FileFormat fileFormat;
+
+  @Parameterized.Parameters
+  public static Object[] parameters() {
+    // TODO add orc and parquet
+    return new Object[] {"avro"};
+  }
+
+  TestFlinkScan(String fileFormat) {
+    this.fileFormat = FileFormat.valueOf(fileFormat.toUpperCase(Locale.ENGLISH));
+  }
+
+  @Before
+  public void before() throws IOException {
+    File warehouseFile = TEMPORARY_FOLDER.newFolder();
+    Assert.assertTrue(warehouseFile.delete());
+    conf = new Configuration();
+    warehouse = "file:" + warehouseFile;
+    catalog = new HadoopCatalog(conf, warehouse);
+  }
+
+  private List<Row> execute(Table table) throws IOException {
+    return executeWithOptions(table, null, null, null, null, null, null, null, null);
+  }
+
+  private List<Row> execute(Table table, List<String> projectFields) throws IOException {
+    return executeWithOptions(table, projectFields, null, null, null, null, null, null, null);
+  }
+
+  protected abstract List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId,
+      Long startSnapshotId, Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter)
+      throws IOException;
+
+  protected abstract void assertResiduals(List<Row> results, List<Record> writeRecords, List<Record> filteredRecords)
+      throws IOException;
+
+  @Test
+  public void testUnpartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 2, 0L);
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testPartitionedTable() throws Exception {
+    Table table = catalog.createTable(TableIdentifier.of("default", "t"), SCHEMA, SPEC);
+    List<Record> expectedRecords = RandomGenericData.generate(SCHEMA, 1, 0L);
+    expectedRecords.get(0).set(2, "2020-03-20");
+    new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(
+        org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords);
+    assertRecords(execute(table), expectedRecords);
+  }
+
+  @Test
+  public void testProjection() throws Exception {

Review comment:
       Another case: Project with a new renamed schema 




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r477005510



##########
File path: flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java
##########
@@ -98,4 +101,26 @@ public static TableSchema toSchema(RowType rowType) {
     }
     return builder.build();
   }
+
+  /**
+   * Project columns from a {@link Schema} using a projected fields.
+   *
+   * @param schema a Schema
+   * @param projectedFields projected fields from Flink
+   * @return a Schema corresponding to the Flink projection
+   */
+  public static Schema projectWithReordering(Schema schema, List<String> projectedFields) {

Review comment:
       Only UUID need be fixed...




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r488106931



##########
File path: flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java
##########
@@ -0,0 +1,114 @@
+/*
+ * 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.api.DataTypes;
+import org.apache.flink.table.api.TableSchema;
+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.RowType;
+import org.apache.flink.table.types.utils.TypeConversions;
+import org.apache.flink.types.Row;
+import org.apache.iceberg.Schema;
+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 FlinkSource.Builder builder;
+
+  public TestFlinkInputFormat(String fileFormat) {
+    super(fileFormat);
+  }
+
+  @Override
+  public void before() throws IOException {
+    super.before();
+    builder = FlinkSource.forBounded().tableLoader(TableLoader.fromHadoopTable(warehouse + "/default/t"));
+  }
+
+  @Override
+  protected List<Row> executeWithOptions(
+      Table table, List<String> projectFields, CatalogLoader loader, Long snapshotId, Long startSnapshotId,
+      Long endSnapshotId, Long asOfTimestamp, List<Expression> filters, String sqlFilter) throws IOException {
+    ScanOptions options = ScanOptions.builder().snapshotId(snapshotId).startSnapshotId(startSnapshotId)
+        .endSnapshotId(endSnapshotId).asOfTimestamp(asOfTimestamp).build();

Review comment:
       I think the tests would be much more readable if the `ScanOptions` builder were used directly. For example, this is hard to understand:
   
   ```java
   assertRecords(executeWithOptions(table, null, null, null, snapshotId1, null, null, null, null), expected1, SCHEMA);
   ```
   
   But you could rewrite that like this:
   ```java
   ScanOptions options = ScanOptions.builder().startSnapshotId(snapshotId1).build();
   assertRecords(executeWithOptions(table, options), expected1, SCHEMA);
   ```
   
   In addition, passing `null` into this leaks the default state within the builder into the tests: test authors need to know that passing `null` for `CatalogLoader` is supported. I think it is better to let the test authors use the builder pattern.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-678112074


   Fine,  I will take a look.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] JingsongLi commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
JingsongLi commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-698682917


   Thanks @rdblue for the review, I'll address your test comments in next PR (Integrate to SQL).
   The reason for a separate `ScanOptions` is: For SQL layer, `ScanOptions` contains options in `CREATE TABLE ... WITH (options)`, while other parameters of `FlinkSource.Builder` are not. This means that the parameters in `ScanOptions` must be in the form of string.
   If you don't think it is necessary, we can also merge it into the builder.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#issuecomment-698666137


   I'm going ahead and merging this because I think the remaining things are minor, but we will probably want to fix them.
   
   First, breaking the tests across two classes could be cleaner. In the future, I'd recommend writing tests and not breaking across a parent/child class until you want to add the other set of tests. That way it is easier to see what is needed and will be clean.
   
   Second, I don't understand the benefit of having a separate `ScanOptions` class, besides that it can pull options out of a map. It seems to me that it would be simpler to just have the `FlinkSource.Builder` class and move all of the scan options to that class.


----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #1346: Flink: Introduce Flink InputFormat

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1346:
URL: https://github.com/apache/iceberg/pull/1346#discussion_r487612733



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java
##########
@@ -0,0 +1,195 @@
+/*
+ * 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.io.UncheckedIOException;
+import java.util.List;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.table.api.TableColumn;
+import org.apache.flink.table.api.TableSchema;
+import org.apache.flink.table.data.RowData;
+import org.apache.flink.table.runtime.typeutils.RowDataTypeInfo;
+import org.apache.flink.table.types.logical.RowType;
+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.FlinkCatalogFactory;
+import org.apache.iceberg.flink.FlinkSchemaUtil;
+import org.apache.iceberg.flink.TableLoader;
+import org.apache.iceberg.hadoop.SerializableConfiguration;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class FlinkSource {
+  private FlinkSource() {
+  }
+
+  /**
+   * Initialize a {@link Builder} to read the data from iceberg table in bounded mode. Reading a snapshot of the table.
+   *
+   * @return {@link Builder} to connect the iceberg table.
+   */
+  public static Builder forBounded() {
+    return new BoundedBuilder();
+  }
+
+  /**
+   * Source builder to build {@link DataStream}.
+   */
+  public abstract static class Builder {
+    private StreamExecutionEnvironment env;
+    private Table table;
+    private TableLoader tableLoader;
+    private List<String> selectedFields;
+    private TableSchema projectedSchema;
+    private ScanOptions options = ScanOptions.builder().build();
+    private List<Expression> filterExpressions;
+    private org.apache.hadoop.conf.Configuration hadoopConf;
+
+    private RowDataTypeInfo rowTypeInfo;
+
+    // -------------------------- Required options -------------------------------
+
+    public Builder tableLoader(TableLoader newLoader) {
+      this.tableLoader = newLoader;
+      return this;
+    }
+
+    // -------------------------- Optional options -------------------------------
+
+    public Builder table(Table newTable) {
+      this.table = newTable;
+      return this;
+    }
+
+    public Builder filters(List<Expression> newFilters) {
+      this.filterExpressions = newFilters;
+      return this;
+    }
+
+    public Builder project(TableSchema schema) {
+      this.projectedSchema = schema;
+      return this;
+    }
+
+    public Builder select(String... fields) {
+      this.selectedFields = Lists.newArrayList(fields);
+      return this;
+    }
+
+    public Builder select(List<String> fields) {
+      this.selectedFields = fields;
+      return this;
+    }
+
+    public Builder options(ScanOptions newOptions) {
+      this.options = newOptions;
+      return this;
+    }
+
+    public Builder hadoopConf(org.apache.hadoop.conf.Configuration newConf) {
+      this.hadoopConf = newConf;
+      return this;
+    }
+
+    public Builder env(StreamExecutionEnvironment newEnv) {
+      this.env = newEnv;
+      return this;
+    }
+
+    StreamExecutionEnvironment getEnv() {
+      return env;
+    }
+
+    RowDataTypeInfo getRowTypeInfo() {
+      return rowTypeInfo;
+    }
+
+    public FlinkInputFormat buildFormat() {
+      Preconditions.checkNotNull(tableLoader, "TableLoader should not be null");
+
+      hadoopConf = hadoopConf == null ? FlinkCatalogFactory.clusterHadoopConf() : hadoopConf;
+
+      Schema icebergSchema;
+      FileIO io;
+      EncryptionManager encryption;
+      if (table == null) {
+        // load required fields by table loader.
+        tableLoader.open(hadoopConf);
+        try (TableLoader loader = tableLoader) {
+          table = loader.loadTable();
+          icebergSchema = table.schema();
+          io = table.io();
+          encryption = table.encryption();
+        } catch (IOException e) {
+          throw new UncheckedIOException(e);
+        }
+      } else {
+        icebergSchema = table.schema();
+        io = table.io();
+        encryption = table.encryption();
+      }
+
+      if (projectedSchema != null && selectedFields != null) {
+        throw new IllegalArgumentException(
+            "Cannot using both requestedSchema and projectedFields to project");
+      }
+
+      TableSchema projectedTableSchema = projectedSchema;
+      TableSchema tableSchema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema));
+      if (selectedFields != null) {
+        TableSchema.Builder builder = TableSchema.builder();
+        for (String field : selectedFields) {
+          TableColumn column = tableSchema.getTableColumn(field).orElseThrow(
+              () -> new IllegalArgumentException(String.format("The field(%s) can not be found in the table schema: %s",
+                  field, tableSchema)));
+          builder.field(column.getName(), column.getType());
+        }
+        projectedTableSchema = builder.build();
+      }
+
+      rowTypeInfo = RowDataTypeInfo.of((RowType) (projectedTableSchema == null ? tableSchema : projectedTableSchema)
+              .toRowDataType().getLogicalType());
+
+      Schema expectedSchema = icebergSchema;
+      if (projectedTableSchema != null) {
+        expectedSchema = FlinkSchemaUtil.convert(icebergSchema, projectedTableSchema);
+      }
+
+      return new FlinkInputFormat(tableLoader, expectedSchema, io, encryption, filterExpressions, options,
+          new SerializableConfiguration(hadoopConf));
+    }
+
+    public abstract DataStream<RowData> build();
+  }
+
+  private static final class BoundedBuilder extends Builder {
+    @Override
+    public DataStream<RowData> build() {
+      Preconditions.checkNotNull(getEnv(), "StreamExecutionEnvironment should not be null");
+      FlinkInputFormat format = buildFormat();
+      return getEnv().createInput(format, getRowTypeInfo());

Review comment:
       Not having `endSnapshot` in `ScanOptions` seems to imply that the scan is unbounded.




----------------------------------------------------------------
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:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org