openinx commented on a change in pull request #2305:
URL: https://github.com/apache/iceberg/pull/2305#discussion_r591410354



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java
##########
@@ -79,6 +83,27 @@ InputFile getInputFile(String location) {
     return inputFiles.get(location);
   }
 
+  public void seek(CheckpointedPosition checkpointedPosition)  {
+    // skip files
+    Preconditions.checkArgument(checkpointedPosition.getOffset() < 
combinedTask.files().size(),

Review comment:
       Nit:   could simplify it as: 
   
   ```java
       Preconditions.checkArgument(checkpointedPosition.getOffset() < 
combinedTask.files().size(),
           "Checkpointed file offset is %s, while CombinedScanTask has %s 
files",
           checkpointedPosition.getOffset(), combinedTask.files().size());
   ```

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/FlinkBulkFormatAdaptor.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.reader;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This adaptor use Flink BulkFormat implementation to read data file.
+ * Note that Flink BulkFormat may not support row deletes like {@link 
RowDataIteratorBulkFormat}
+ */
+public class FlinkBulkFormatAdaptor<T> implements BulkFormat<T, 
IcebergSourceSplit> {
+
+  private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+  private final TypeInformation<T> producedType;
+
+  public FlinkBulkFormatAdaptor(Map<FileFormat, BulkFormat<T, 
FileSourceSplit>> bulkFormatProvider) {
+    this.bulkFormatProvider = bulkFormatProvider;
+    // validate that all BulkFormat produce the same type
+    List<TypeInformation<T>> uniqueTypes = bulkFormatProvider.values().stream()
+        .map(bulkFormat -> bulkFormat.getProducedType())
+        .distinct()
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(uniqueTypes.size() == 1,
+        "BulkFormats have the different producedType: " + uniqueTypes);
+    producedType = uniqueTypes.get(0);
+  }
+
+  @Override
+  public Reader<T> createReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, false);
+  }
+
+  @Override
+  public Reader<T> restoreReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, true);
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+
+  @Override
+  public TypeInformation<T> getProducedType() {
+    return producedType;
+  }
+
+  private static final class ReaderAdaptor<T> implements BulkFormat.Reader<T> {
+
+    private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+    private final Configuration config;
+    private final Iterator<FileScanTask> fileIterator;
+    private final boolean isRestored;
+
+    // file offset in CombinedScanTask
+    private int fileOffset = -1;
+    private Reader<T> currentReader;
+
+    ReaderAdaptor(
+        Map<FileFormat, BulkFormat<T, FileSourceSplit>> bulkFormatProvider,
+        Configuration config,
+        IcebergSourceSplit icebergSplit,
+        boolean isRestored) throws IOException {
+      this.config = config;
+      this.bulkFormatProvider = bulkFormatProvider;
+      this.fileIterator = icebergSplit.task().files().iterator();
+      this.isRestored = isRestored;
+
+      final CheckpointedPosition position = 
icebergSplit.checkpointedPosition();
+      if (position != null) {
+        // skip files based on offset in checkpointed position
+        Preconditions.checkArgument(position.getOffset() < 
icebergSplit.task().files().size(),

Review comment:
       Nit:  use the method 
   
   ```java
   checkArgument(boolean expression, @Nullable String errorMessageTemplate, 
@Nullable Object... errorMessageArgs)
   ```

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.reader;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Map;
+import org.apache.flink.api.connector.source.SourceReaderContext;
+import 
org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.iceberg.flink.source.IcebergSourceEvents;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.flink.source.split.MutableIcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+
+public class IcebergSourceReader<T> extends

Review comment:
       This class don't have to be introduced in this PR ? I see there's no 
usage.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/FlinkBulkFormatAdaptor.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.reader;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This adaptor use Flink BulkFormat implementation to read data file.
+ * Note that Flink BulkFormat may not support row deletes like {@link 
RowDataIteratorBulkFormat}
+ */
+public class FlinkBulkFormatAdaptor<T> implements BulkFormat<T, 
IcebergSourceSplit> {
+
+  private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+  private final TypeInformation<T> producedType;
+
+  public FlinkBulkFormatAdaptor(Map<FileFormat, BulkFormat<T, 
FileSourceSplit>> bulkFormatProvider) {
+    this.bulkFormatProvider = bulkFormatProvider;
+    // validate that all BulkFormat produce the same type
+    List<TypeInformation<T>> uniqueTypes = bulkFormatProvider.values().stream()
+        .map(bulkFormat -> bulkFormat.getProducedType())
+        .distinct()
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(uniqueTypes.size() == 1,
+        "BulkFormats have the different producedType: " + uniqueTypes);
+    producedType = uniqueTypes.get(0);
+  }
+
+  @Override
+  public Reader<T> createReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, false);

Review comment:
       Nit:  `new ReaderAdaptor<>(...)` ? 

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/IcebergSourceOptions.java
##########
@@ -0,0 +1,33 @@
+/*
+ * 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.configuration.ConfigOption;
+import org.apache.flink.configuration.ConfigOptions;
+
+public interface IcebergSourceOptions {

Review comment:
       We've introduced a `FlinkTableOptions` , I think it's not friendly to 
create options classes for source, sink, table etc.  Maybe we could rename the 
`FlinkTableOptions` to `FlinkConfigOptions`, and put all the options into that 
class.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/FlinkBulkFormatAdaptor.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.reader;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This adaptor use Flink BulkFormat implementation to read data file.
+ * Note that Flink BulkFormat may not support row deletes like {@link 
RowDataIteratorBulkFormat}
+ */
+public class FlinkBulkFormatAdaptor<T> implements BulkFormat<T, 
IcebergSourceSplit> {
+
+  private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+  private final TypeInformation<T> producedType;
+
+  public FlinkBulkFormatAdaptor(Map<FileFormat, BulkFormat<T, 
FileSourceSplit>> bulkFormatProvider) {
+    this.bulkFormatProvider = bulkFormatProvider;
+    // validate that all BulkFormat produce the same type
+    List<TypeInformation<T>> uniqueTypes = bulkFormatProvider.values().stream()
+        .map(bulkFormat -> bulkFormat.getProducedType())
+        .distinct()
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(uniqueTypes.size() == 1,
+        "BulkFormats have the different producedType: " + uniqueTypes);
+    producedType = uniqueTypes.get(0);
+  }
+
+  @Override
+  public Reader<T> createReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, false);
+  }
+
+  @Override
+  public Reader<T> restoreReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, true);
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+
+  @Override
+  public TypeInformation<T> getProducedType() {
+    return producedType;
+  }
+
+  private static final class ReaderAdaptor<T> implements BulkFormat.Reader<T> {
+
+    private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+    private final Configuration config;
+    private final Iterator<FileScanTask> fileIterator;
+    private final boolean isRestored;
+
+    // file offset in CombinedScanTask
+    private int fileOffset = -1;
+    private Reader<T> currentReader;
+
+    ReaderAdaptor(
+        Map<FileFormat, BulkFormat<T, FileSourceSplit>> bulkFormatProvider,
+        Configuration config,
+        IcebergSourceSplit icebergSplit,
+        boolean isRestored) throws IOException {
+      this.config = config;
+      this.bulkFormatProvider = bulkFormatProvider;
+      this.fileIterator = icebergSplit.task().files().iterator();
+      this.isRestored = isRestored;
+
+      final CheckpointedPosition position = 
icebergSplit.checkpointedPosition();
+      if (position != null) {
+        // skip files based on offset in checkpointed position
+        Preconditions.checkArgument(position.getOffset() < 
icebergSplit.task().files().size(),
+            String.format("Checkpointed file offset is %d, while 
CombinedScanTask has %d files",
+                position.getOffset(), icebergSplit.task().files().size()));
+        for (int i = 0; i < position.getOffset(); ++i) {
+          fileIterator.next();
+          fileOffset++;
+        }
+        // first file may need to skip records
+        setupReader(position.getRecordsAfterOffset());
+      } else {
+        setupReader(0L);
+      }
+    }
+
+    /**
+     * TODO: we can't return RecordIterator with empty data.
+     * Otherwise, caller may assume it is end of input.
+     * We probably need to add a {@code hasNext()} API to
+     * {@link RecordIterator} to achieve the goal.
+     */
+    @Nullable
+    @Override
+    public RecordIterator<T> readBatch() throws IOException {
+      RecordIterator<T> result = null;
+      while (currentReader != null || fileIterator.hasNext()) {
+        if (currentReader == null) {
+          setupReader(0L);
+        } else {
+          result = currentReader.readBatch();
+          if (result != null) {
+            break;
+          } else {
+            closeCurrentReader();
+          }
+        }
+      }
+      if (result == null) {
+        return null;
+      } else {
+        return new RecordIteratorAdaptor(fileOffset, result);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      closeCurrentReader();
+    }
+
+    private void closeCurrentReader() throws IOException {
+      if (currentReader != null) {
+        currentReader.close();
+        currentReader = null;
+      }
+    }
+
+    private void setupReader(long skipRecordCount) throws IOException {
+      if (fileIterator.hasNext()) {
+        final FileScanTask fileScanTask = fileIterator.next();
+        final FileFormat fileFormat = fileScanTask.file().format();
+        if (!bulkFormatProvider.containsKey(fileFormat)) {
+          throw new IOException("Unsupported file format: " + fileFormat);
+        }
+        final BulkFormat<T, FileSourceSplit> bulkFormat = 
bulkFormatProvider.get(fileFormat);
+        fileOffset++;
+        final FileSourceSplit fileSourceSplit = new FileSourceSplit(
+            "",
+            new Path(URI.create(fileScanTask.file().path().toString())),

Review comment:
       So the  `FileSourceSplit` will use the flink's fs interface to access 
the underlying files ?   We iceberg currently has our own `FileIO` interface,  
the object storage services are implementing this interface to write/read data 
to cloud.  If we introduce flink fs here ,  I'm concerning that we have to 
implement both flink fs interfaces and iceberg FileIO interfaces for making the 
experimental unified source work.

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/FlinkBulkFormatAdaptor.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.reader;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This adaptor use Flink BulkFormat implementation to read data file.
+ * Note that Flink BulkFormat may not support row deletes like {@link 
RowDataIteratorBulkFormat}
+ */
+public class FlinkBulkFormatAdaptor<T> implements BulkFormat<T, 
IcebergSourceSplit> {
+
+  private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+  private final TypeInformation<T> producedType;
+
+  public FlinkBulkFormatAdaptor(Map<FileFormat, BulkFormat<T, 
FileSourceSplit>> bulkFormatProvider) {
+    this.bulkFormatProvider = bulkFormatProvider;
+    // validate that all BulkFormat produce the same type
+    List<TypeInformation<T>> uniqueTypes = bulkFormatProvider.values().stream()
+        .map(bulkFormat -> bulkFormat.getProducedType())
+        .distinct()
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(uniqueTypes.size() == 1,
+        "BulkFormats have the different producedType: " + uniqueTypes);
+    producedType = uniqueTypes.get(0);
+  }
+
+  @Override
+  public Reader<T> createReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, false);
+  }
+
+  @Override
+  public Reader<T> restoreReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, true);

Review comment:
       ditto

##########
File path: 
flink/src/main/java/org/apache/iceberg/flink/source/reader/FlinkBulkFormatAdaptor.java
##########
@@ -0,0 +1,222 @@
+/*
+ * 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.reader;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import javax.annotation.Nullable;
+import org.apache.flink.api.common.typeinfo.TypeInformation;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.connector.file.src.FileSourceSplit;
+import org.apache.flink.connector.file.src.reader.BulkFormat;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.flink.connector.file.src.util.MutableRecordAndPosition;
+import org.apache.flink.connector.file.src.util.RecordAndPosition;
+import org.apache.flink.core.fs.Path;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * This adaptor use Flink BulkFormat implementation to read data file.
+ * Note that Flink BulkFormat may not support row deletes like {@link 
RowDataIteratorBulkFormat}
+ */
+public class FlinkBulkFormatAdaptor<T> implements BulkFormat<T, 
IcebergSourceSplit> {
+
+  private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+  private final TypeInformation<T> producedType;
+
+  public FlinkBulkFormatAdaptor(Map<FileFormat, BulkFormat<T, 
FileSourceSplit>> bulkFormatProvider) {
+    this.bulkFormatProvider = bulkFormatProvider;
+    // validate that all BulkFormat produce the same type
+    List<TypeInformation<T>> uniqueTypes = bulkFormatProvider.values().stream()
+        .map(bulkFormat -> bulkFormat.getProducedType())
+        .distinct()
+        .collect(Collectors.toList());
+    Preconditions.checkArgument(uniqueTypes.size() == 1,
+        "BulkFormats have the different producedType: " + uniqueTypes);
+    producedType = uniqueTypes.get(0);
+  }
+
+  @Override
+  public Reader<T> createReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, false);
+  }
+
+  @Override
+  public Reader<T> restoreReader(Configuration config, IcebergSourceSplit 
split) throws IOException {
+    return new ReaderAdaptor<T>(bulkFormatProvider, config, split, true);
+  }
+
+  @Override
+  public boolean isSplittable() {
+    return false;
+  }
+
+  @Override
+  public TypeInformation<T> getProducedType() {
+    return producedType;
+  }
+
+  private static final class ReaderAdaptor<T> implements BulkFormat.Reader<T> {
+
+    private final Map<FileFormat, BulkFormat<T, FileSourceSplit>> 
bulkFormatProvider;
+    private final Configuration config;
+    private final Iterator<FileScanTask> fileIterator;
+    private final boolean isRestored;
+
+    // file offset in CombinedScanTask
+    private int fileOffset = -1;
+    private Reader<T> currentReader;
+
+    ReaderAdaptor(
+        Map<FileFormat, BulkFormat<T, FileSourceSplit>> bulkFormatProvider,
+        Configuration config,
+        IcebergSourceSplit icebergSplit,
+        boolean isRestored) throws IOException {
+      this.config = config;
+      this.bulkFormatProvider = bulkFormatProvider;
+      this.fileIterator = icebergSplit.task().files().iterator();
+      this.isRestored = isRestored;
+
+      final CheckpointedPosition position = 
icebergSplit.checkpointedPosition();
+      if (position != null) {
+        // skip files based on offset in checkpointed position
+        Preconditions.checkArgument(position.getOffset() < 
icebergSplit.task().files().size(),
+            String.format("Checkpointed file offset is %d, while 
CombinedScanTask has %d files",
+                position.getOffset(), icebergSplit.task().files().size()));
+        for (int i = 0; i < position.getOffset(); ++i) {
+          fileIterator.next();
+          fileOffset++;
+        }
+        // first file may need to skip records
+        setupReader(position.getRecordsAfterOffset());
+      } else {
+        setupReader(0L);
+      }
+    }
+
+    /**
+     * TODO: we can't return RecordIterator with empty data.
+     * Otherwise, caller may assume it is end of input.
+     * We probably need to add a {@code hasNext()} API to
+     * {@link RecordIterator} to achieve the goal.
+     */
+    @Nullable
+    @Override
+    public RecordIterator<T> readBatch() throws IOException {
+      RecordIterator<T> result = null;
+      while (currentReader != null || fileIterator.hasNext()) {
+        if (currentReader == null) {
+          setupReader(0L);
+        } else {
+          result = currentReader.readBatch();
+          if (result != null) {
+            break;
+          } else {
+            closeCurrentReader();
+          }
+        }
+      }
+      if (result == null) {
+        return null;
+      } else {
+        return new RecordIteratorAdaptor(fileOffset, result);
+      }
+    }
+
+    @Override
+    public void close() throws IOException {
+      closeCurrentReader();
+    }
+
+    private void closeCurrentReader() throws IOException {
+      if (currentReader != null) {
+        currentReader.close();
+        currentReader = null;
+      }
+    }
+
+    private void setupReader(long skipRecordCount) throws IOException {
+      if (fileIterator.hasNext()) {
+        final FileScanTask fileScanTask = fileIterator.next();
+        final FileFormat fileFormat = fileScanTask.file().format();
+        if (!bulkFormatProvider.containsKey(fileFormat)) {
+          throw new IOException("Unsupported file format: " + fileFormat);
+        }
+        final BulkFormat<T, FileSourceSplit> bulkFormat = 
bulkFormatProvider.get(fileFormat);
+        fileOffset++;
+        final FileSourceSplit fileSourceSplit = new FileSourceSplit(
+            "",
+            new Path(URI.create(fileScanTask.file().path().toString())),
+            fileScanTask.start(),
+            fileScanTask.length(),
+            new String[0],
+            // Since this is always for a single data file and some format
+            // (like ParquetVectorizedInputFormat) requires NO_OFFSET,
+            // we just always set the file offset to NO_OFFSET.
+            new CheckpointedPosition(CheckpointedPosition.NO_OFFSET, 
skipRecordCount));
+        if (isRestored) {
+          currentReader = bulkFormat.restoreReader(config, fileSourceSplit);
+        } else {
+          currentReader = bulkFormat.createReader(config, fileSourceSplit);
+        }
+      } else {
+        closeCurrentReader();
+      }
+    }
+  }
+
+  private static final class RecordIteratorAdaptor<T> implements 
RecordIterator<T> {
+
+    private final long fileOffset;
+    private final RecordIterator<T> iterator;
+    private final MutableRecordAndPosition mutableRecordAndPosition;

Review comment:
       Nit:  use `MutableRecordAndPosition<T>` 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:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to