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/19 07:53:26 UTC

[GitHub] [iceberg] SreeramGarlapati opened a new pull request #2611: [WorkInProgress] Spark3 structured streaming micro_batch read support

SreeramGarlapati opened a new pull request #2611:
URL: https://github.com/apache/iceberg/pull/2611


   Am creating this PR as a starting point on my way to collaborate and build on Spark3 structured streaming read support for Iceberg table with @aokolnychyi & @RussellSpitzer. Truly apologize - if I am violating any policies of this repository. Please let me know & I will remove this PR.
   
   This work is an extension of Spark2 work done in PR #2272 - only that - this is for Spark3.
   
   StreamingOffset was defined in PR #2092 using `Offset` class defined in Spark2. But, Spark3 introduces another base class for Offsets. As a result I had to copy this.
   
   Hit a roadblock - while trying to model MicroBatchStream scan & seeking guidance.


-- 
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] SreeramGarlapati commented on a change in pull request #2611: [WorkInProgress] Spark3 structured streaming micro_batch read support

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -159,8 +159,14 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, caseSensitive, schemaWithMetadataColumns(), filterExpressions, options);
+    // TODO: understand how to differentiate that this is a spark streaming microbatch scan.
+    if (false) {

Review comment:
       @aokolnychyi / @RussellSpitzer 
   Spark3 gives ScanBuilder - abstraction - to define all types of Scans (Batch, MicroBatch & Continuous). But, the current implementation / class modelling - has SparkBatchScan as the Scan implementation.
   Looking at some of the concerns of BatchScan - all the way from the State maintenance of a single SnapshotId to read from, the asOfTimeStamp & features like VectorizedReads - all of these don't seem relevant to Streaming Scans.
   So, I feel that we need to divide out Streaming Scans into a different class.
   Does this thought process - make sense?
   If we go by this route - do you folks know - how to pass different Scan objects to Spark based on Batch vs Streaming?




-- 
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] SreeramGarlapati closed pull request #2611: Spark3 structured streaming micro_batch read support

Posted by GitBox <gi...@apache.org>.
SreeramGarlapati closed pull request #2611:
URL: https://github.com/apache/iceberg/pull/2611


   


-- 
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] SreeramGarlapati commented on a change in pull request #2611: Spark3 structured streaming micro_batch read support

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



##########
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.util.List;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+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.MicroBatches.MicroBatchBuilder;
+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.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.OffsetSeq;
+import org.apache.spark.sql.execution.streaming.OffsetSeqLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+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 int batchSize;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetSeqLog offsetSeqLog;
+
+  private StreamingOffset initialOffset = null;
+  private PlannedEndOffset previousEndOffset = 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.batchSize = Spark3Util.batchSize(table.properties(), options);
+    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.offsetSeqLog = checkpointLocation != null ?
+        new OffsetSeqLog(spark, getOffsetLogLocation(checkpointLocation)) :
+        null;
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    if (isTableEmpty()) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    StreamingOffset microBatchStartOffset = isFirstBatch() ? initialOffset : previousEndOffset;
+    if (isEndOfSnapshot(microBatchStartOffset)) {
+      microBatchStartOffset = getNextAvailableSnapshot(microBatchStartOffset);
+    }
+
+    previousEndOffset = calculateEndOffset(microBatchStartOffset);
+    return previousEndOffset;
+  }
+
+  @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 PlannedEndOffset,
+        "The end offset passed to planInputPartitions() is not the one that is returned by lastOffset()");
+    PlannedEndOffset endOffset = (PlannedEndOffset) end;
+
+    List<FileScanTask> fileScanTasks = endOffset.getMicroBatch().tasks();
+
+    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 (isInitialOffsetResolved()) {
+      return initialOffset;
+    }
+
+    if (isStreamResumedFromCheckpoint()) {
+      initialOffset = calculateInitialOffsetFromCheckpoint();
+      return initialOffset;
+    }
+
+    List<Long> snapshotIds = SnapshotUtil.currentAncestors(table);
+    if (snapshotIds.isEmpty()) {
+      initialOffset = StreamingOffset.START_OFFSET;
+      Preconditions.checkState(isTableEmpty(),
+          "criteria behind isTableEmpty() changed.");
+    } else {
+      initialOffset = new StreamingOffset(Iterables.getLast(snapshotIds), 0, true);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private String getOffsetLogLocation(String checkpointLocation) {

Review comment:
       @aokolnychyi @RussellSpitzer @rdblue @holdenk - do you folks know of a better way to read the last checkpointed offset. 
   Am needing this **hack** for cases where spark cluster goes down and has to restart stream from an old checkpoint. I definitely DO NOT plan to Keep this HACK. Am trying to understand a better way to do this. Truly appreciate any help 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] SreeramGarlapati commented on a change in pull request #2611: Spark3 structured streaming micro_batch read support

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



##########
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.util.List;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+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.MicroBatches.MicroBatchBuilder;
+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.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.OffsetSeq;
+import org.apache.spark.sql.execution.streaming.OffsetSeqLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+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 int batchSize;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetSeqLog offsetSeqLog;
+
+  private StreamingOffset initialOffset = null;
+  private PlannedEndOffset previousEndOffset = 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.batchSize = Spark3Util.batchSize(table.properties(), options);
+    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.offsetSeqLog = checkpointLocation != null ?
+        new OffsetSeqLog(spark, getOffsetLogLocation(checkpointLocation)) :
+        null;
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    if (isTableEmpty()) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    StreamingOffset microBatchStartOffset = isFirstBatch() ? initialOffset : previousEndOffset;
+    if (isEndOfSnapshot(microBatchStartOffset)) {
+      microBatchStartOffset = getNextAvailableSnapshot(microBatchStartOffset);
+    }
+
+    previousEndOffset = calculateEndOffset(microBatchStartOffset);
+    return previousEndOffset;
+  }
+
+  @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 PlannedEndOffset,
+        "The end offset passed to planInputPartitions() is not the one that is returned by lastOffset()");
+    PlannedEndOffset endOffset = (PlannedEndOffset) end;
+
+    List<FileScanTask> fileScanTasks = endOffset.getMicroBatch().tasks();
+
+    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 (isInitialOffsetResolved()) {
+      return initialOffset;
+    }
+
+    if (isStreamResumedFromCheckpoint()) {
+      initialOffset = calculateInitialOffsetFromCheckpoint();
+      return initialOffset;
+    }
+
+    List<Long> snapshotIds = SnapshotUtil.currentAncestors(table);
+    if (snapshotIds.isEmpty()) {
+      initialOffset = StreamingOffset.START_OFFSET;
+      Preconditions.checkState(isTableEmpty(),
+          "criteria behind isTableEmpty() changed.");
+    } else {
+      initialOffset = new StreamingOffset(Iterables.getLast(snapshotIds), 0, true);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private String getOffsetLogLocation(String checkpointLocation) {
+    return new Path(checkpointLocation.replace("/sources/0", ""), "offsets").toString();
+  }
+
+  private boolean isInitialOffsetResolved() {
+    return initialOffset != null;
+  }
+
+  private StreamingOffset calculateInitialOffsetFromCheckpoint() {
+    Preconditions.checkState(isStreamResumedFromCheckpoint(),
+        "Stream is not resumed from checkpoint.");
+
+    OffsetSeq offsetSeq = offsetSeqLog.getLatest().get()._2;
+
+    List<Option<Offset>> offsetSeqCol = JavaConverters.seqAsJavaList(offsetSeq.offsets());
+    Option<Offset> optionalOffset = offsetSeqCol.get(0);
+
+    StreamingOffset checkpointedOffset = StreamingOffset.fromJson(optionalOffset.get().json());
+    return checkpointedOffset;
+  }
+
+  private boolean isStreamResumedFromCheckpoint() {
+    Preconditions.checkState(!isInitialOffsetResolved(),
+        "isStreamResumedFromCheckpoint() is invoked without resolving initialOffset");
+
+    return offsetSeqLog != null && offsetSeqLog.getLatest() != null && offsetSeqLog.getLatest().isDefined();
+  }
+
+  private boolean isFirstBatch() {
+    return previousEndOffset == null || previousEndOffset.equals(StreamingOffset.START_OFFSET);
+  }
+
+  private boolean isTableEmpty() {
+    Preconditions.checkState(isInitialOffsetResolved(),
+        "isTableEmpty() is invoked without resolving initialOffset");
+
+    return initialOffset.equals(StreamingOffset.START_OFFSET);
+  }
+
+  private StreamingOffset getNextAvailableSnapshot(StreamingOffset microBatchStartOffset) {
+    if (table.currentSnapshot().snapshotId() == microBatchStartOffset.snapshotId()) {
+      return microBatchStartOffset;
+    }
+
+    Snapshot previousSnapshot = table.snapshot(microBatchStartOffset.snapshotId());
+    Snapshot pointer = table.currentSnapshot();
+    while (pointer != null && previousSnapshot.snapshotId() != pointer.parentId()) {
+      Preconditions.checkState(pointer.operation().equals(DataOperations.APPEND),

Review comment:
       TODO: Add unittest coverage for overwrite operation.




-- 
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] SreeramGarlapati commented on a change in pull request #2611: Spark3 structured streaming micro_batch read support

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



##########
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.util.List;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+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.MicroBatches.MicroBatchBuilder;
+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.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.OffsetSeq;
+import org.apache.spark.sql.execution.streaming.OffsetSeqLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+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 int batchSize;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetSeqLog offsetSeqLog;
+
+  private StreamingOffset initialOffset = null;
+  private PlannedEndOffset previousEndOffset = 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.batchSize = Spark3Util.batchSize(table.properties(), options);
+    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.offsetSeqLog = checkpointLocation != null ?
+        new OffsetSeqLog(spark, getOffsetLogLocation(checkpointLocation)) :
+        null;
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    if (isTableEmpty()) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    StreamingOffset microBatchStartOffset = isFirstBatch() ? initialOffset : previousEndOffset;
+    if (isEndOfSnapshot(microBatchStartOffset)) {
+      microBatchStartOffset = getNextAvailableSnapshot(microBatchStartOffset);
+    }
+
+    previousEndOffset = calculateEndOffset(microBatchStartOffset);
+    return previousEndOffset;
+  }
+
+  @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 PlannedEndOffset,
+        "The end offset passed to planInputPartitions() is not the one that is returned by lastOffset()");
+    PlannedEndOffset endOffset = (PlannedEndOffset) end;
+
+    List<FileScanTask> fileScanTasks = endOffset.getMicroBatch().tasks();
+
+    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 (isInitialOffsetResolved()) {
+      return initialOffset;
+    }
+
+    if (isStreamResumedFromCheckpoint()) {
+      initialOffset = calculateInitialOffsetFromCheckpoint();
+      return initialOffset;
+    }
+
+    List<Long> snapshotIds = SnapshotUtil.currentAncestors(table);
+    if (snapshotIds.isEmpty()) {
+      initialOffset = StreamingOffset.START_OFFSET;
+      Preconditions.checkState(isTableEmpty(),
+          "criteria behind isTableEmpty() changed.");
+    } else {
+      initialOffset = new StreamingOffset(Iterables.getLast(snapshotIds), 0, true);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private String getOffsetLogLocation(String checkpointLocation) {

Review comment:
       @aokolnychyi @RussellSpitzer @rdblue @holdenk - do you folks know of a better way to read the last checkpointed offset. 
   Am needing this hack for cases where spark cluster goes down and has to restart stream from an old checkpoint. I definitely DO NOT plan to Keep this HACK. Am trying to understand a better way to do this. Truly appreciate any help 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] SreeramGarlapati commented on pull request #2611: Spark3 structured streaming micro_batch read support

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


   @holdenk - thanks for your first take on the PR. Would be happy to hear out https://github.com/viirya's thoughts as well. Am unable to tag https://github.com/viirya. Please see if you can...
   
   The PR description about GDPR - is to decide between 
   a) whether to ignore deletes by `default` or 
   b) whether to take a special flag to be able to ignore deletes. 
   
   The reasoning is that - 
   * In Spark Structured streaming - we are STREAMING Iceberg table - row by row. ==> So, there is NO way to STREAM deletes from Iceberg table. 
   * Which implies ==> when we encounter deletes - we are left with 2 choices
       1. fail - with `UnsupportedSnapshotDataOperation - DELETE`
       2. Ignore deletes and move on. 
   * almost all of the users of the Iceberg tables want to be GDPR compliant.
   * which implies ==> they would want to delete some rows out of their Iceberg table & want to stream reads out of that table.
   * So, if we throw - UnsupportedOperation - when we encountered a Snapshot of type Delete while reading off of Iceberg Table -  potentially all tables out there will need to handle this!
   * So, my proposal is to accept that - Iceberg tables will have GDPR deletes - i.e., - if the Iceberg table has Snapshots which are marked as Delete - we will ignore that Snapshot for streaming read purposes. In the later PRs I will expose a Spark Option - which will give the ability to fail the streaming read - if a Delete is encountered. 
   
   Did this make sense!? Happy to discuss.


-- 
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] SreeramGarlapati commented on pull request #2611: Spark3 structured streaming micro_batch read support

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


   > From the PR description though, I'm not sure I understand the logic behind skipping deletes since GDPR compliance is rather important.
   
   Hi @holdenk - after gaving it a bit more thought - I actually think otherwise. Overall, I agree with you. Ignoring deletes from tables might spook out some of the consumers of this data. For now, I removed the IgnoreDeletes part from the PR and updated the description to reflect the same here: https://github.com/apache/iceberg/pull/2660
   
   PS: I was playing around with this branch to squash-merge my 28 commits to one commit (as it was very chatty) & in the process git closed this PR and is not letting me reopen this. So, I had to create a brand new 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] holdenk commented on pull request #2611: Spark3 structured streaming micro_batch read support

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


   Ok this is great work, I'm going to have to get back up to speed on the streaming stuff. Perhaps https://github.com/viirya has some thoughts here as well.
   From the PR description though, I'm not sure I understand the logic behind skipping deletes since GDPR compliance is rather important.


-- 
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] SreeramGarlapati commented on a change in pull request #2611: Spark3 structured streaming micro_batch read support

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



##########
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.util.List;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+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.MicroBatches.MicroBatchBuilder;
+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.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.OffsetSeq;
+import org.apache.spark.sql.execution.streaming.OffsetSeqLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+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 int batchSize;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetSeqLog offsetSeqLog;
+
+  private StreamingOffset initialOffset = null;
+  private PlannedEndOffset previousEndOffset = 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.batchSize = Spark3Util.batchSize(table.properties(), options);
+    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.offsetSeqLog = checkpointLocation != null ?
+        new OffsetSeqLog(spark, getOffsetLogLocation(checkpointLocation)) :
+        null;
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    if (isTableEmpty()) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    StreamingOffset microBatchStartOffset = isFirstBatch() ? initialOffset : previousEndOffset;
+    if (isEndOfSnapshot(microBatchStartOffset)) {
+      microBatchStartOffset = getNextAvailableSnapshot(microBatchStartOffset);
+    }
+
+    previousEndOffset = calculateEndOffset(microBatchStartOffset);
+    return previousEndOffset;
+  }
+
+  @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 PlannedEndOffset,
+        "The end offset passed to planInputPartitions() is not the one that is returned by lastOffset()");
+    PlannedEndOffset endOffset = (PlannedEndOffset) end;
+
+    List<FileScanTask> fileScanTasks = endOffset.getMicroBatch().tasks();
+
+    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 (isInitialOffsetResolved()) {
+      return initialOffset;
+    }
+
+    if (isStreamResumedFromCheckpoint()) {
+      initialOffset = calculateInitialOffsetFromCheckpoint();
+      return initialOffset;
+    }
+
+    List<Long> snapshotIds = SnapshotUtil.currentAncestors(table);
+    if (snapshotIds.isEmpty()) {
+      initialOffset = StreamingOffset.START_OFFSET;
+      Preconditions.checkState(isTableEmpty(),
+          "criteria behind isTableEmpty() changed.");
+    } else {
+      initialOffset = new StreamingOffset(Iterables.getLast(snapshotIds), 0, true);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private String getOffsetLogLocation(String checkpointLocation) {

Review comment:
       @aokolnychyi @RussellSpitzer @rdblue @holdenk - do you folks know of a better way to read the last checkpointed offset. 
   Am needing this **hack** for cases where spark cluster goes down and has to restart stream from an old checkpoint. I definitely DO NOT plan to Keep this HACK. Am trying to understand a better way to do this. Truly appreciate any help here.
   [Started a conversation in iceberg slack channel](https://the-asf.slack.com/archives/CF01LKV9S/p1622529032058200).




-- 
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] SreeramGarlapati commented on a change in pull request #2611: Spark3 structured streaming micro_batch read support

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



##########
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.util.List;
+import java.util.Optional;
+import org.apache.hadoop.fs.Path;
+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.MicroBatches.MicroBatchBuilder;
+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.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.OffsetSeq;
+import org.apache.spark.sql.execution.streaming.OffsetSeqLog;
+import org.apache.spark.sql.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.collection.JavaConverters;
+
+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 int batchSize;
+  private final Long splitSize;
+  private final Integer splitLookback;
+  private final Long splitOpenFileCost;
+  private final boolean localityPreferred;
+  private final OffsetSeqLog offsetSeqLog;
+
+  private StreamingOffset initialOffset = null;
+  private PlannedEndOffset previousEndOffset = 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.batchSize = Spark3Util.batchSize(table.properties(), options);
+    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.offsetSeqLog = checkpointLocation != null ?
+        new OffsetSeqLog(spark, getOffsetLogLocation(checkpointLocation)) :
+        null;
+  }
+
+  @Override
+  public Offset latestOffset() {
+    initialOffset();
+
+    if (isTableEmpty()) {
+      return StreamingOffset.START_OFFSET;
+    }
+
+    StreamingOffset microBatchStartOffset = isFirstBatch() ? initialOffset : previousEndOffset;
+    if (isEndOfSnapshot(microBatchStartOffset)) {
+      microBatchStartOffset = getNextAvailableSnapshot(microBatchStartOffset);
+    }
+
+    previousEndOffset = calculateEndOffset(microBatchStartOffset);
+    return previousEndOffset;
+  }
+
+  @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 PlannedEndOffset,
+        "The end offset passed to planInputPartitions() is not the one that is returned by lastOffset()");
+    PlannedEndOffset endOffset = (PlannedEndOffset) end;
+
+    List<FileScanTask> fileScanTasks = endOffset.getMicroBatch().tasks();
+
+    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 (isInitialOffsetResolved()) {
+      return initialOffset;
+    }
+
+    if (isStreamResumedFromCheckpoint()) {
+      initialOffset = calculateInitialOffsetFromCheckpoint();
+      return initialOffset;
+    }
+
+    List<Long> snapshotIds = SnapshotUtil.currentAncestors(table);
+    if (snapshotIds.isEmpty()) {
+      initialOffset = StreamingOffset.START_OFFSET;
+      Preconditions.checkState(isTableEmpty(),
+          "criteria behind isTableEmpty() changed.");
+    } else {
+      initialOffset = new StreamingOffset(Iterables.getLast(snapshotIds), 0, true);
+    }
+
+    return initialOffset;
+  }
+
+  @Override
+  public Offset deserializeOffset(String json) {
+    return StreamingOffset.fromJson(json);
+  }
+
+  @Override
+  public void commit(Offset end) {
+  }
+
+  @Override
+  public void stop() {
+  }
+
+  private String getOffsetLogLocation(String checkpointLocation) {

Review comment:
       @aokolnychyi @RussellSpitzer @rdblue @holdenk - do you folks know of a better way to read the last checkpointed offset. This is needed for cases where spark cluster goes down and has to restart stream from an old checkpoint.




-- 
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] SreeramGarlapati commented on a change in pull request #2611: [WorkInProgress] Spark3 structured streaming micro_batch read support

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



##########
File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkScanBuilder.java
##########
@@ -159,8 +159,14 @@ private Schema schemaWithMetadataColumns() {
 
   @Override
   public Scan build() {
-    return new SparkBatchQueryScan(
-        spark, table, caseSensitive, schemaWithMetadataColumns(), filterExpressions, options);
+    // TODO: understand how to differentiate that this is a spark streaming microbatch scan.
+    if (false) {

Review comment:
       @aokolnychyi / @RussellSpitzer / @holdenk 
   Spark3 gives ScanBuilder - abstraction - to define all types of Scans (Batch, MicroBatch & Continuous). But, the current implementation / class modelling - has SparkBatchScan as the Scan implementation.
   Looking at some of the concerns of BatchScan - all the way from the State maintenance of a single SnapshotId to read from, the asOfTimeStamp & features like VectorizedReads - all of these don't seem relevant to Streaming Scans.
   So, I feel that we need to divide out Streaming Scans into a different class.
   Does this thought process - make sense?
   If we go by this route - do you folks know - how to pass different Scan objects to Spark based on Batch vs Streaming?




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