jerryshao commented on a change in pull request #796: [WIP] Support Spark Structured Streaming Read for Iceberg URL: https://github.com/apache/incubator-iceberg/pull/796#discussion_r383089813
########## File path: spark/src/main/java/org/apache/iceberg/spark/source/StreamingReader.java ########## @@ -0,0 +1,223 @@ +/* + * 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 com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.iceberg.CombinedScanTask; +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.util.SnapshotUtil; +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; + +public class StreamingReader extends Reader implements MicroBatchReader { + private static final Logger LOG = LoggerFactory.getLogger(StreamingReader.class); + + private Offset startOffset; + private Offset endOffset; + + private final Table table; + private final int maxTasksPerTrigger; + + static class SnapshotTasks { + private final long snapshotId; + private final List<CombinedScanTask> tasks; + private final int size; + + @VisibleForTesting + SnapshotTasks(long snapshotId, List<CombinedScanTask> tasks) { + this.snapshotId = snapshotId; + this.tasks = tasks; + this.size = tasks.size(); + } + + long snapshotId() { + return snapshotId; + } + + List<CombinedScanTask> tasks() { + return tasks; + } + + int size() { + return size; + } + } + + StreamingReader(Table table, Broadcast<FileIO> io, Broadcast<EncryptionManager> encryptionManager, + boolean caseSensitive, DataSourceOptions options) { + super(table, io, encryptionManager, caseSensitive, options); + + this.table = table; + this.maxTasksPerTrigger = + options.get("max-tasks-per-trigger").map(Integer::parseInt).orElse(1000); + Preconditions.checkArgument(this.maxTasksPerTrigger > 0, + "Option max-tasks-per-trigger should > 0"); + } + + @Override + public void setOffsetRange(Optional<Offset> start, Optional<Offset> end) { + startOffset = start.orElse(calculateInitialOffset()); + endOffset = Optional.ofNullable(startOffset) + .map(s -> end.orElse(calculateEndOffset((StreamingOffset) s))) + .orElse(null); + } + + @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) {} + + @Override + public void stop() {} + + @Override + protected List<CombinedScanTask> tasks() { + Preconditions.checkState(startOffset != null, + "Start offset is null when planning tasks, this probably because there's no snapshot in the table currently"); + Preconditions.checkState(endOffset != null, "End offset is null when planning tasks"); + + StreamingOffset start = (StreamingOffset) startOffset; Review comment: Fixed. ---------------------------------------------------------------- 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: us...@infra.apache.org With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org For additional commands, e-mail: issues-h...@iceberg.apache.org