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



##########
File path: 
spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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 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.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.io.FileIO;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+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.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.util.CaseInsensitiveStringMap;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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 StreamingOffset initialOffset;
+
+  SparkMicroBatchStream(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);
+
+    long tableSplitSize = PropertyUtil.propertyAsLong(table.properties(), 
SPLIT_SIZE, SPLIT_SIZE_DEFAULT);
+    this.splitSize = Spark3Util.propertyAsLong(options, 
SparkReadOptions.SPLIT_SIZE, tableSplitSize);
+
+    int tableSplitLookback = PropertyUtil.propertyAsInt(table.properties(), 
SPLIT_LOOKBACK, SPLIT_LOOKBACK_DEFAULT);
+    this.splitLookback = Spark3Util.propertyAsInt(options, 
SparkReadOptions.LOOKBACK, tableSplitLookback);
+
+    long tableSplitOpenFileCost = PropertyUtil.propertyAsLong(
+        table.properties(), SPLIT_OPEN_FILE_COST, 
SPLIT_OPEN_FILE_COST_DEFAULT);
+    this.splitOpenFileCost = Spark3Util.propertyAsLong(options, 
SPLIT_OPEN_FILE_COST, tableSplitOpenFileCost);
+
+    InitialOffsetStore initialOffsetStore = 
InitialOffsetStore.getInstance(table, checkpointLocation);
+    this.initialOffset = getOrWriteInitialOffset(initialOffsetStore);
+  }
+
+  @Override
+  public Offset latestOffset() {

Review comment:
       That was [my initial 
approach](https://github.com/apache/iceberg/pull/2660/commits/41041f3ac7b00edf0e2ffb0308bdfec211a523d8#diff-bce275bc6d503b0d12fe345b3df8838da3f4ba208b2a0667c47b9ac0e62e22bc).
 But then, there was **correctness issue** - in the case of `Trigger.Once`. 
Seemslike [there is precedence 
](https://databricks.com/blog/2017/05/22/running-streaming-jobs-day-10x-cost-savings.html)
 in using `Trigger.Once` in the context of automating Batch processing - where 
`readStream.Trigger(Trigger.Once)` is expected to return **all events** - that 
the table currently has. Pl. let me know if I am able to put the point across - 
happy to go over a call and explain.
   
   Due to the above reason - **we CANNOT incrementally emit the 
`latestOffsets`** - unless the user **EXPLICITLY** expresses intent to return 
less amount of data than what is present in the table - for ex: via an option 
like `size`.
   
   & then, evaluating the concern that @aokolnychyi's & you (@rdblue's) raised; 
one particular place that I empathised is - ppl will try this new Streaming on 
their existing tables - which in most of the usecases - will be very large 
tables & will run into the problem of Spark cluster not being able to handle 
their first micro_batch. bad experience!
   
   Looking at the rate limiting change - it looks like a big change. There are 
already way too many things to agree on - in the current PR. After carefully 
reviewing the changes required for implementing rate-limiting - given the 
amount changes - I strongly feel that - we must first get this change IN - with 
that limitation on the first batch and then iterate - rate limiting on it. I 
will prioritize rate limiting effort - immediately after this PR goes in. 
@rdblue / @aokolnychyi - does this make sense?




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