rdblue commented on a change in pull request #796: Support Spark Structured 
Streaming Read for Iceberg
URL: https://github.com/apache/incubator-iceberg/pull/796#discussion_r392427126
 
 

 ##########
 File path: 
spark/src/main/java/org/apache/iceberg/spark/source/StreamingReader.java
 ##########
 @@ -0,0 +1,298 @@
+/*
+ * 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.spark.source;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Streams;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.sources.v2.DataSourceOptions;
+import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader;
+import org.apache.spark.sql.sources.v2.reader.streaming.Offset;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A mirco-batch based Spark Structured Streaming reader for Iceberg table. It 
will track the added
+ * files and generate tasks per batch to process newly added files. By default 
it will process
+ * all the newly added files to the current snapshot in each batch, user could 
also set this
+ * configuration "max-files-per-trigger" to control the number of files 
processed per batch.
+ */
+class StreamingReader extends Reader implements MicroBatchReader {
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamingReader.class);
+  private static final int DEFAULT_MAX_FILES_PER_TRIGGER = 1000;
+
+  private static final Comparator<FileScanTask> COMPARATOR = (left, right) -> {
+    int pathCompare = 
left.file().path().toString().compareTo(right.file().path().toString());
+    int startCompare = Long.compare(left.start(), right.start());
+    int lengthCompare = Long.compare(left.length(), right.length());
+    return pathCompare == 0 ? (startCompare == 0 ? lengthCompare : 
startCompare) : pathCompare;
+  };
+
+  private StreamingOffset startOffset;
+  private StreamingOffset endOffset;
+
+  private final Table table;
+  private final int maxFilesPerTrigger;
+  private final long splitSize;
+  private final int lookback;
+  private final long openFileCost;
+  private final Long startSnapshotId;
+
+  /**
+   * Utility class to track the snapshotId, task and index of task within this 
snapshot.
+   */
+  @VisibleForTesting
+  static class IndexedTask {
+    private final long snapshotId;
+    private final FileScanTask task;
+    private final int index;
+
+    IndexedTask(long snapshotId, FileScanTask task, int index) {
+      this.snapshotId = snapshotId;
+      this.task = task;
+      this.index = index;
+    }
+
+    long snapshotId() {
+      return snapshotId;
+    }
+
+    FileScanTask task() {
+      return task;
+    }
+
+    int index() {
+      return index;
+    }
+  }
+
+  StreamingReader(Table table, Broadcast<FileIO> io, 
Broadcast<EncryptionManager> encryptionManager,
+                  boolean caseSensitive, DataSourceOptions options) {
+    super(table, io, encryptionManager, caseSensitive, options);
+
+    this.table = table;
+    this.maxFilesPerTrigger =
+      
options.get("max-files-per-trigger").map(Integer::parseInt).orElse(DEFAULT_MAX_FILES_PER_TRIGGER);
+    Preconditions.checkArgument(maxFilesPerTrigger > 0,
+        "Option max-files-per-trigger '%d' should > 0", maxFilesPerTrigger);
+
+    this.startSnapshotId = 
options.get("starting-snapshot-id").map(Long::parseLong).orElse(null);
+    if (startSnapshotId != null) {
+      if (!SnapshotUtil.ancestorOf(table, 
table.currentSnapshot().snapshotId(), startSnapshotId)) {
+        throw new IllegalStateException("The option starting-snapshot-id " + 
startSnapshotId +
+          "is not a valid snapshot id");
+      }
+    }
+
+    this.splitSize = options.get("split-size").map(Long::parseLong).orElse(
+        Optional.ofNullable(table.properties().get(TableProperties.SPLIT_SIZE))
+            .map(Long::parseLong)
+            .orElse(TableProperties.SPLIT_SIZE_DEFAULT));
+    this.lookback = options.get("lookback").map(Integer::parseInt).orElse(
+        
Optional.ofNullable(table.properties().get(TableProperties.SPLIT_LOOKBACK))
+            .map(Integer::parseInt)
+            .orElse(TableProperties.SPLIT_LOOKBACK_DEFAULT));
+    this.openFileCost = 
options.get("file-open-cost").map(Long::parseLong).orElse(
+        
Optional.ofNullable(table.properties().get(TableProperties.SPLIT_OPEN_FILE_COST))
+            .map(Long::parseLong)
+            .orElse(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT));
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public void setOffsetRange(Optional<Offset> start, Optional<Offset> end) {
+    table.refresh();
+
+    if (start.isPresent() && 
!StreamingOffset.START_OFFSET.equals(start.get())) {
+      this.startOffset = (StreamingOffset) start.get();
+      this.endOffset = (StreamingOffset) 
end.orElse(calculateEndOffset(startOffset));
+    } else {
+      // If starting offset is "START_OFFSET" (there's no snapshot in the last 
batch), or starting
+      // offset is not set, then we need to calculate the starting offset 
again.
+      this.startOffset = calculateStartingOffset();
+      this.endOffset = calculateEndOffset(startOffset);
+    }
+  }
+
+  @Override
+  public Offset getStartOffset() {
+    if (startOffset == null) {
+      throw new IllegalStateException("Start offset is not set");
+    }
+
+    return startOffset;
+  }
+
+  @Override
+  public Offset getEndOffset() {
+    if (endOffset == null) {
+      throw new IllegalStateException("End offset is not set");
+    }
+
+    return endOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+    // Since all the data and metadata of Iceberg is as it is, nothing needs 
to commit when
+    // offset is processed, so no need to implement this method.
+  }
+
+  @Override
+  public void stop() {}
+
+  @Override
+  @SuppressWarnings("unchecked")
+  protected List<CombinedScanTask> tasks() {
+    LOG.info("Processing data from {} exclusive to {} inclusive", startOffset, 
endOffset);
 
 Review comment:
   Minor: Can you move this to after the check for whether there is work to do?
   
   It would also be good to add an info message if there are no tasks that says 
there are no 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:
us...@infra.apache.org


With regards,
Apache Git Services

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

Reply via email to