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 2021/05/14 03:59:26 UTC

[GitHub] [iceberg] stevenzwu commented on a change in pull request #2305: Flink: FLIP-27 source split and reader

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



##########
File path: flink/src/main/java/org/apache/iceberg/flink/source/CombinedScanTaskIterator.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.Iterator;
+import javax.annotation.Nullable;
+import org.apache.flink.connector.file.src.util.CheckpointedPosition;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterator;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Base class of Flink iterators.
+ *
+ * @param <T> is the Java class returned by this iterator whose objects contain one or more rows.
+ */
+public class CombinedScanTaskIterator<T> implements CloseableIterator<T> {
+
+  private final FileIteratorReader<T> fileIteratorReader;
+  private final DecryptedInputFiles decryptedInputFiles;
+  private final Iterator<FileScanTask> tasks;
+  private final Position position;
+
+  private CloseableIterator<T> currentIterator;
+
+  public CombinedScanTaskIterator(CombinedScanTask combinedTask, FileIO io, EncryptionManager encryption,
+                                  FileIteratorReader<T> fileIteratorReader) {
+    this(combinedTask, io, encryption, fileIteratorReader, null);
+  }
+
+  public CombinedScanTaskIterator(CombinedScanTask combinedTask, FileIO io, EncryptionManager encryption,
+                                  FileIteratorReader<T> fileIteratorReader, @Nullable Position startingPosition) {
+    this.fileIteratorReader = fileIteratorReader;
+    this.decryptedInputFiles = new DecryptedInputFiles(combinedTask, io, encryption);
+    this.tasks = combinedTask.files().iterator();
+
+    if (startingPosition != null) {
+      this.position = startingPosition;
+      // skip files
+      Preconditions.checkArgument(position.fileOffset() < combinedTask.files().size(),
+          String.format("Starting file offset is %d, while CombinedScanTask has %d files",
+              position.fileOffset(), combinedTask.files().size()));
+      for (long i = 0L; i < position.fileOffset; ++i) {
+        tasks.next();
+      }
+    } else {
+      this.position = new Position();
+    }
+
+    final FileScanTask startingFileTask = tasks.next();
+    this.currentIterator = fileIteratorReader.open(startingFileTask, decryptedInputFiles);
+
+    // skip records
+    for (int i = 0; i < position.recordOffset(); ++i) {
+      if (currentIterator.hasNext()) {
+        currentIterator.next();
+      } else {
+        throw new IllegalArgumentException(String.format(
+            "File has less than %d records: %s", position.recordOffset, startingFileTask.file().path()));
+      }
+    }
+  }
+
+  @Override
+  public boolean hasNext() {
+    updateCurrentIterator();
+    return currentIterator.hasNext();
+  }
+
+  @Override
+  public T next() {
+    updateCurrentIterator();
+    position.advanceRecord();
+    return currentIterator.next();
+  }
+
+  public boolean isCurrentIteratorDone() {
+    return !currentIterator.hasNext();
+  }
+
+  /**
+   * 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 = fileIteratorReader.open(tasks.next(), decryptedInputFiles);
+        position.advanceFile();
+      }
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    // close the current iterator
+    currentIterator.close();
+  }
+
+  public Position position() {
+    return position;
+  }
+
+  public static class Position {

Review comment:
       @openinx following up your comment from the uber PR: https://github.com/apache/iceberg/pull/2105/files#r630834205.
   
   The reason I introduce this mutable `Position` class is to avoid the construction of a <fileOffset, recordOffset> object. It is the current cursor for the iterator. 
   
   Didn't track the recordOffset inside the `FileIteratorReader` for the same reason. Otherwise, `position()` getter will construct a new object each time.
   
   We can't use CheckpointedPosition from Flink for two reasons: (1) it is immutable (2) we want to return the current position (not necessarily CheckpointedPosition).
   




-- 
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