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/11 19:42:07 UTC

[GitHub] [iceberg] holdenk commented on a change in pull request #2272: Support structured streaming read for Iceberg

holdenk commented on a change in pull request #2272:
URL: https://github.com/apache/iceberg/pull/2272#discussion_r630407298



##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/StreamingReader.java
##########
@@ -0,0 +1,370 @@
+/*
+ * 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.Collections;
+import java.util.List;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.stream.Collectors;
+import org.apache.commons.collections.CollectionUtils;
+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.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.encryption.EncryptionManager;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.io.FileIO;
+import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+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.SparkReadOptions;
+import org.apache.iceberg.util.Pair;
+import org.apache.iceberg.util.PropertyUtil;
+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;
+
+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;
+
+/**
+ * A micro-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 String MAX_SIZE_PER_BATCH = "max-size-per-batch";
+  private static final String START_SNAPSHOT_ID = "start-snapshot-id";
+
+  private StreamingOffset startOffset;
+  private StreamingOffset endOffset;
+
+  private final Table table;
+  private final long maxSizePerBatch;
+  private final Long startSnapshotId;
+  private final long splitSize;
+  private final int splitLookback;
+  private final long splitOpenFileCost;
+  private Boolean readUsingBatch = null;
+
+  /**
+   * Used to cache the pending batches for this streaming batch interval.
+   */
+  private Pair<StreamingOffset, List<MicroBatch>> cachedPendingBatches = null;
+
+  StreamingReader(Table table, Broadcast<FileIO> io, Broadcast<EncryptionManager> encryptionManager,
+      boolean caseSensitive, DataSourceOptions options) {
+    super(table, io, encryptionManager, caseSensitive, options);
+
+    this.table = table;
+    this.maxSizePerBatch = options.get(MAX_SIZE_PER_BATCH).map(Long::parseLong).orElse(Long.MAX_VALUE);
+    Preconditions.checkArgument(maxSizePerBatch > 0L,
+        "Option max-size-per-batch '%s' should > 0", maxSizePerBatch);
+
+    this.startSnapshotId = options.get(START_SNAPSHOT_ID).map(Long::parseLong).orElse(null);
+    if (startSnapshotId != null) {
+      if (!SnapshotUtil.ancestorOf(table, table.currentSnapshot().snapshotId(), startSnapshotId)) {
+        throw new IllegalArgumentException("The option start-snapshot-id " + startSnapshotId +
+            " is not an ancestor of the current snapshot");
+      }
+    }
+
+    long tableSplitSize = Optional.ofNullable(splitSize())
+        .orElseGet(
+            () -> PropertyUtil.propertyAsLong(table.properties(), SPLIT_SIZE, SPLIT_SIZE_DEFAULT));
+    int tableSplitLookback = Optional.ofNullable(splitLookback())
+        .orElseGet(() -> PropertyUtil
+            .propertyAsInt(table.properties(), SPLIT_LOOKBACK, SPLIT_LOOKBACK_DEFAULT));
+    long tableSplitOpenFileCost = Optional.ofNullable(splitOpenFileCost())
+        .orElseGet(() -> PropertyUtil.propertyAsLong(table.properties(), SPLIT_OPEN_FILE_COST,
+            SPLIT_OPEN_FILE_COST_DEFAULT));
+
+    this.splitSize = options.getLong(SparkReadOptions.SPLIT_SIZE, tableSplitSize);
+    this.splitLookback = options.getInt(SparkReadOptions.LOOKBACK, tableSplitLookback);
+    this.splitOpenFileCost = options
+        .getLong(SparkReadOptions.FILE_OPEN_COST, tableSplitOpenFileCost);
+  }
+
+  @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.orElseGet(() -> 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) {

Review comment:
       Would it make sense to clean up the cachedPendingBatches here, or would they already have been removed at this point?

##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -127,6 +130,23 @@ public StreamWriter createStreamWriter(String runId, StructType dsStruct,
     return new StreamingWriter(table, io, encryptionManager, options, queryId, mode, appId, writeSchema, dsStruct);
   }
 
+  @Override
+  public MicroBatchReader createMicroBatchReader(Optional<StructType> schema, String checkpointLocation,

Review comment:
       So it seems like we're not using the checkpointLocation here, do we not need to store anything to be able to recover on failure? I think this is the case because as we read data it's not like it gets "consumed" but I just want to make sure.

##########
File path: spark2/src/main/java/org/apache/iceberg/spark/source/IcebergSource.java
##########
@@ -127,6 +130,23 @@ public StreamWriter createStreamWriter(String runId, StructType dsStruct,
     return new StreamingWriter(table, io, encryptionManager, options, queryId, mode, appId, writeSchema, dsStruct);
   }
 
+  @Override
+  public MicroBatchReader createMicroBatchReader(Optional<StructType> schema, String checkpointLocation,
+                                                 DataSourceOptions options) {
+    if (schema.isPresent()) {
+      throw new IllegalStateException("Iceberg does not support specifying the schema at read time");
+    }
+
+    Configuration conf = new Configuration(lazyBaseConf());
+    Table table = getTableAndResolveHadoopConfiguration(options, conf);
+    String caseSensitive = lazySparkSession().conf().get("spark.sql.caseSensitive");

Review comment:
       Maybe add a comment about why this is resolved from the session conf instead of the merged session conf / source options?




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