SreeramGarlapati commented on a change in pull request #2660: URL: https://github.com/apache/iceberg/pull/2660#discussion_r648582742
########## File path: spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java ########## @@ -0,0 +1,312 @@ +/* + * 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.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.SnapshotSummary; +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.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 InitialOffsetStore initialOffsetStore; + private final StreamingOffset initialOffset; + + SparkMicroBatchStream(SparkSession spark, + Table table, boolean caseSensitive, Schema expectedSchema, + CaseInsensitiveStringMap options, String checkpointLocation) { + this.sparkContext = JavaSparkContext.fromSparkContext(spark.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.initialOffsetStore = InitialOffsetStore.getInstance(spark, checkpointLocation); + this.initialOffset = getOrWriteInitialOffset(table, initialOffsetStore); Review comment: @holdenk - thinking this thru - I don't see a way out of using `getOrWriteInitialOffset`. Here's why: for ex: a table has snapshots S1, S2 & here are the series of events that will happen: 1 -> start a `readStream` on the table 2 -> at which point the connector should return - start offset as [S1, 0, scanAllFiles = true] - last offset as [S2, lastFilePosition, scanAllFiles = false] 3 -> `readStream` completes processing this MicroBatch and **no NEW** events came to the table. 4 -> now lets say that the spark cluster running this `readStream` - died and restarted. 5 -> when it died - also - there is an additional Snapshot S3 - that was written to the iceberg table. 6 -> and Snapshots S1 and S2 are deleted - due to retention policy - as these 2 Snapshots expired. 6 -> Now, when the spark cluster came back up and **re-started** the `readStream` on this iceberg table - we need to infer that - **before the snapshot S3 - this stream already read some events from the table and hence - we need to set `scanAllFiles` in snapshot S3 to false.** - **without storing the fact that - the stream started from S1 - there is just no way to implement this!** - if we don't persist where the stream started (the `initialOffset`) - then we will have to assume that table's current starting offset (i.e., S3 in this case) - might be the initial offset. In the above case `S3` is clearly not the initialOffset for the actual readStream; & as a result, we will end up setting `scanAllFiles` - to true! - which will result in **replaying all the data from S1 and S2** - which **breaks the contract of stream**. pl. let me know if this made sense. I am happy to go on a call and explain or hear any other possible solutions. -- 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]
