rdblue commented on a change in pull request #2660:
URL: https://github.com/apache/iceberg/pull/2660#discussion_r646917922



##########
File path: 
spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
##########
@@ -0,0 +1,309 @@
+/*
+ * 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 java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStream;
+import java.io.OutputStreamWriter;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Optional;
+import org.apache.iceberg.CombinedScanTask;
+import org.apache.iceberg.DataOperations;
+import org.apache.iceberg.FileScanTask;
+import org.apache.iceberg.MicroBatches;
+import org.apache.iceberg.MicroBatches.MicroBatch;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.SchemaParser;
+import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.io.CharStreams;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkBatchScan.ReadTask;
+import org.apache.iceberg.spark.source.SparkBatchScan.ReaderFactory;
+import org.apache.iceberg.util.PropertyUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.broadcast.Broadcast;
+import org.apache.spark.sql.SparkSession;
+import org.apache.spark.sql.connector.read.InputPartition;
+import org.apache.spark.sql.connector.read.PartitionReaderFactory;
+import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
+import org.apache.spark.sql.connector.read.streaming.Offset;
+import org.apache.spark.sql.execution.streaming.HDFSMetadataLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.reflect.ClassTag;
+
+import static org.apache.iceberg.TableProperties.SPLIT_LOOKBACK;
+import static org.apache.iceberg.TableProperties.SPLIT_LOOKBACK_DEFAULT;
+import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST;
+import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT;
+import static org.apache.iceberg.TableProperties.SPLIT_SIZE;
+import static org.apache.iceberg.TableProperties.SPLIT_SIZE_DEFAULT;
+
+public class SparkMicroBatchStream implements MicroBatchStream {
+  private static final Logger LOG = 
LoggerFactory.getLogger(SparkMicroBatchStream.class);
+
+  private final JavaSparkContext sparkContext;
+  private final Table table;
+  private final boolean caseSensitive;
+  private final Schema expectedSchema;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetLog offsetLog;
+
+  private StreamingOffset initialOffset = null;
+
+  SparkMicroBatchStream(SparkSession spark, JavaSparkContext sparkContext,
+                        Table table, boolean caseSensitive, Schema 
expectedSchema,
+                        CaseInsensitiveStringMap options, String 
checkpointLocation) {
+    this.sparkContext = sparkContext;
+    this.table = table;
+    this.caseSensitive = caseSensitive;
+    this.expectedSchema = expectedSchema;
+    this.localityPreferred = Spark3Util.isLocalityEnabled(table.io(), 
table.location(), options);
+    this.splitSize = Optional.ofNullable(Spark3Util.propertyAsLong(options, 
SparkReadOptions.SPLIT_SIZE, null))
+        .orElseGet(() -> PropertyUtil.propertyAsLong(table.properties(), 
SPLIT_SIZE, SPLIT_SIZE_DEFAULT));
+    this.splitLookback = Optional.ofNullable(Spark3Util.propertyAsInt(options, 
SparkReadOptions.LOOKBACK, null))
+        .orElseGet(() -> PropertyUtil.propertyAsInt(table.properties(), 
SPLIT_LOOKBACK, SPLIT_LOOKBACK_DEFAULT));
+    this.splitOpenFileCost = Optional.ofNullable(
+        Spark3Util.propertyAsLong(options, SparkReadOptions.FILE_OPEN_COST, 
null))
+        .orElseGet(() -> PropertyUtil.propertyAsLong(table.properties(), 
SPLIT_OPEN_FILE_COST,
+            SPLIT_OPEN_FILE_COST_DEFAULT));
+    this.offsetLog = OffsetLog.getInstance(spark, checkpointLocation);
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    final Snapshot latestSnapshot = table.currentSnapshot();
+    if (latestSnapshot == null) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    return new StreamingOffset(
+        latestSnapshot.snapshotId(),
+        Iterables.size(latestSnapshot.addedFiles()),
+        latestSnapshot.snapshotId() == initialOffset.snapshotId());
+  }
+
+  @Override
+  public InputPartition[] planInputPartitions(Offset start, Offset end) {
+    if (end.equals(StreamingOffset.START_OFFSET)) {
+      return new InputPartition[0];
+    }
+
+    // broadcast the table metadata as input partitions will be sent to 
executors
+    Broadcast<Table> tableBroadcast = 
sparkContext.broadcast(SerializableTable.copyOf(table));
+    String expectedSchemaString = SchemaParser.toJson(expectedSchema);
+
+    Preconditions.checkState(
+        end instanceof StreamingOffset,
+        "The end offset passed to planInputPartitions() is not an instance of 
StreamingOffset.");
+
+    Preconditions.checkState(
+        start instanceof StreamingOffset,
+        "The start offset passed to planInputPartitions() is not an instance 
of StreamingOffset.");
+
+    StreamingOffset endOffset = (StreamingOffset) end;
+    StreamingOffset startOffset = (StreamingOffset) start;
+
+    List<FileScanTask> fileScanTasks = getFileScanTasks(startOffset, 
endOffset);
+
+    CloseableIterable<FileScanTask> splitTasks = TableScanUtil.splitFiles(
+        CloseableIterable.withNoopClose(fileScanTasks),
+        splitSize);
+    List<CombinedScanTask> combinedScanTasks = Lists.newArrayList(
+        TableScanUtil.planTasks(splitTasks, splitSize, splitLookback, 
splitOpenFileCost));
+    InputPartition[] readTasks = new InputPartition[combinedScanTasks.size()];
+
+    for (int i = 0; i < combinedScanTasks.size(); i++) {
+      readTasks[i] = new ReadTask(
+          combinedScanTasks.get(i), tableBroadcast, expectedSchemaString,
+          caseSensitive, localityPreferred);
+    }
+
+    return readTasks;
+  }
+
+  @Override
+  public PartitionReaderFactory createReaderFactory() {
+    int batchSizeValueToDisableColumnarReads = 0;
+    return new ReaderFactory(batchSizeValueToDisableColumnarReads);
+  }
+
+  @Override
+  public Offset initialOffset() {
+    if (initialOffset != null) {
+      return initialOffset;
+    }
+
+    if (offsetLog.isOffsetLogInitialized()) {
+      initialOffset = offsetLog.getLatest();
+      return initialOffset;
+    }
+
+    if (table.currentSnapshot() == null) {
+      initialOffset = StreamingOffset.START_OFFSET;
+    } else {
+      initialOffset = new StreamingOffset(
+          SnapshotUtil.oldestSnapshot(table).snapshotId(), 0, true);
+      this.offsetLog.addInitialOffset(initialOffset);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private List<FileScanTask> getFileScanTasks(StreamingOffset startOffset, 
StreamingOffset endOffset) {
+    final List<FileScanTask> fileScanTasks = new ArrayList<>();
+    MicroBatch latestMicroBatch = null;
+    do {
+      final StreamingOffset currentOffset =
+          latestMicroBatch != null && latestMicroBatch.lastIndexOfSnapshot() ?
+          getNextAvailableSnapshot(latestMicroBatch.snapshotId()) :
+          startOffset;
+
+      latestMicroBatch = 
MicroBatches.from(table.snapshot(currentOffset.snapshotId()), table.io())
+          .caseSensitive(caseSensitive)
+          .specsById(table.specs())
+          .generate(currentOffset.position(), Long.MAX_VALUE, 
currentOffset.shouldScanAllFiles());
+
+      fileScanTasks.addAll(latestMicroBatch.tasks());
+    } while (latestMicroBatch.snapshotId() != endOffset.snapshotId());
+
+    return fileScanTasks;
+  }
+
+  private StreamingOffset getNextAvailableSnapshot(long snapshotId) {
+    Snapshot previousSnapshot = table.snapshot(snapshotId);
+    Snapshot pointer = table.currentSnapshot();
+    while (pointer != null && previousSnapshot.snapshotId() != 
pointer.parentId()) {
+      
Preconditions.checkState(pointer.operation().equals(DataOperations.APPEND),
+          "Encountered Snapshot DataOperation other than APPEND.");
+
+      pointer = table.snapshot(pointer.parentId());
+    }
+
+    Preconditions.checkState(pointer != null,
+        "snapshot on which the stream operated has been garbage collected.");
+
+    return new StreamingOffset(pointer.snapshotId(), 0L, false);
+  }
+
+  interface OffsetLog {
+    static OffsetLog getInstance(SparkSession spark, String 
checkpointLocation) {
+      return new OffsetLogImpl(spark, checkpointLocation);
+    }
+
+    void addInitialOffset(StreamingOffset offset);
+
+    boolean isOffsetLogInitialized();
+
+    StreamingOffset getLatest();
+  }
+
+  private static class OffsetLogImpl implements OffsetLog {
+    private final IcebergSourceOffsetLog offsetSeqLog;
+
+    OffsetLogImpl(SparkSession spark, String checkpointLocation) {
+      this.offsetSeqLog = checkpointLocation != null ?
+          new IcebergSourceOffsetLog(spark, checkpointLocation) :
+          null;
+    }
+
+    @Override
+    public void addInitialOffset(StreamingOffset offset) {
+      this.offsetSeqLog.add(0, offset);
+    }
+
+    @Override
+    public boolean isOffsetLogInitialized() {
+      return offsetSeqLog != null &&
+          offsetSeqLog.getLatest() != null &&
+          offsetSeqLog.getLatest().isDefined();
+    }
+
+    @Override
+    public StreamingOffset getLatest() {
+      return offsetSeqLog.getLatest().get()._2;
+    }
+  }
+
+  private static class IcebergSourceOffsetLog extends 
HDFSMetadataLog<StreamingOffset> {
+    private final String path;
+
+    IcebergSourceOffsetLog(SparkSession sparkSession, String path) {
+      super(sparkSession, path, ClassTag.apply(StreamingOffset.class));
+      this.path = path;
+    }
+
+    @Override
+    public void serialize(StreamingOffset metadata, OutputStream out) {
+      try {
+        BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(out, 
StandardCharsets.UTF_8));
+        writer.write(metadata.json());
+        writer.flush();
+      } catch (IOException ioException) {
+        throw new IllegalArgumentException(
+            String.format("Failed to deserialize latest checkpoint from: %s, 
with error: %s.", path, ioException));
+      }
+    }
+
+    @Override
+    public StreamingOffset deserialize(InputStream in) {
+      try {
+        String content = CharStreams.toString(new InputStreamReader(in, 
StandardCharsets.UTF_8));

Review comment:
       We're trying not to expand our use of Guava because it needs to be 
shaded and can change between versions. I think there should be an easy 
non-Guava way to do this. Can you try that instead?




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