-
Notifications
You must be signed in to change notification settings - Fork 2.9k
Spark3 structured streaming micro_batch read support #2660
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
Merged
Changes from all commits
Commits
Show all changes
40 commits
Select commit
Hold shift + click to select a range
41041f3
Spark3 structured streaming micro_batch read support
SreeramGarlapati 51c9e87
integrate with spark3 checkpointing.
SreeramGarlapati 1b7dbb0
remove guava dependency
SreeramGarlapati 5a1020f
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati 6f57b55
fix NPE in SnapshotUtil
SreeramGarlapati 6fe5657
rename OffsetLog to InitialOffsetStore
SreeramGarlapati 15efe95
optimization: use snapshot summary to get number of added files inste…
SreeramGarlapati 9c9b4de
handle the case when read stream on iceberg table source restarts fro…
SreeramGarlapati b4acade
refactor test code
SreeramGarlapati cab843b
mark initialOffset final
SreeramGarlapati 5e59082
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati fa85859
refresh table metadata before computing offsets.
SreeramGarlapati 96aaa22
remove dependency on HDFSMetadataLog for checkpointing
SreeramGarlapati f286cee
reduce SparkScan.ReaderFactory visibility from public to package-private
SreeramGarlapati 2ac1269
fix latestOffset value when latestOffset is the startingOffset and st…
SreeramGarlapati 7ed9781
rename getFilesScanTasks to calculateFilesScanTasks
SreeramGarlapati 6ee958c
remove unused instance variable - spark
SreeramGarlapati c101a9e
refactor SparkMicroBatchStream constructor
SreeramGarlapati ce16a76
remove unused log variable.
SreeramGarlapati baa03d6
remove unused imports.
SreeramGarlapati 78e5bdb
replace the optimization to use snapshot.summary with data file enume…
SreeramGarlapati 00fe477
checkstyle: remove unused imports
SreeramGarlapati cb4f200
confirm to codebase error message fmt's
SreeramGarlapati 33bf3c5
refactor initialOffsetStore
SreeramGarlapati 353f274
remove dependency on Hadoop's path - use SLASH instead
SreeramGarlapati f22a587
change the default behavior scanning the first snapshot of spark3 str…
SreeramGarlapati 75e8430
Merge branch 'master' of https://github.com/apache/iceberg into spark…
SreeramGarlapati 3c60ef8
rename variable in SnapshotUtil.snapshotAfter from pointer to current
SreeramGarlapati d671d5e
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati 344ed1f
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati f33b26e
fix the javadoc of methods added to SnapshotUtil
SreeramGarlapati 9b2fd4e
refactor SnapshotUtil.snapshotAfter
SreeramGarlapati cc7c887
Add streaming read tests for Catalog based tables
daksha121 da2528b
checkstyle
SreeramGarlapati fadd15c
replace usage of Streams with Iterables
SreeramGarlapati 8bb4048
add comment to explain how streaming stop and resume is simulated
SreeramGarlapati 12c675a
revert the usage of 'delete from' statement to the original appraoach…
SreeramGarlapati 507f7d8
remove redundant creation of sparkSession object
SreeramGarlapati 2baebb1
Remove unnecessary rules from tests
rdblue 7467c0c
Remove unused imports.
rdblue File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
251 changes: 251 additions & 0 deletions
251
spark3/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,251 @@ | ||
| /* | ||
| * 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.io.UncheckedIOException; | ||
| import java.nio.charset.StandardCharsets; | ||
| import java.util.List; | ||
| 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.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.Joiner; | ||
| 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.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 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 Joiner SLASH = Joiner.on("/"); | ||
|
|
||
| private final Table table; | ||
| private final boolean caseSensitive; | ||
| private final String expectedSchema; | ||
| private final Broadcast<Table> tableBroadcast; | ||
| 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.table = table; | ||
| this.caseSensitive = caseSensitive; | ||
| this.expectedSchema = SchemaParser.toJson(expectedSchema); | ||
| this.localityPreferred = Spark3Util.isLocalityEnabled(table.io(), table.location(), options); | ||
| this.tableBroadcast = sparkContext.broadcast(SerializableTable.copyOf(table)); | ||
|
|
||
| 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 = new InitialOffsetStore(table, checkpointLocation); | ||
| this.initialOffset = initialOffsetStore.initialOffset(); | ||
| } | ||
|
|
||
| @Override | ||
| public Offset latestOffset() { | ||
SreeramGarlapati marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| table.refresh(); | ||
| Snapshot latestSnapshot = table.currentSnapshot(); | ||
| if (latestSnapshot == null) { | ||
| return StreamingOffset.START_OFFSET; | ||
| } | ||
|
|
||
| return new StreamingOffset(latestSnapshot.snapshotId(), Iterables.size(latestSnapshot.addedFiles()), false); | ||
| } | ||
|
|
||
| @Override | ||
| public InputPartition[] planInputPartitions(Offset start, Offset end) { | ||
| Preconditions.checkArgument(end instanceof StreamingOffset, "Invalid end offset: %s is not a StreamingOffset", end); | ||
| Preconditions.checkArgument( | ||
| start instanceof StreamingOffset, "Invalid start offset: %s is not a StreamingOffset", start); | ||
|
|
||
| if (end.equals(StreamingOffset.START_OFFSET)) { | ||
| return new InputPartition[0]; | ||
| } | ||
|
|
||
| StreamingOffset endOffset = (StreamingOffset) end; | ||
| StreamingOffset startOffset = (StreamingOffset) start; | ||
|
|
||
| List<FileScanTask> fileScanTasks = planFiles(startOffset, endOffset); | ||
|
|
||
| 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, expectedSchema, | ||
| caseSensitive, localityPreferred); | ||
| } | ||
|
|
||
| return readTasks; | ||
| } | ||
|
|
||
| @Override | ||
| public PartitionReaderFactory createReaderFactory() { | ||
| return new ReaderFactory(0); | ||
| } | ||
|
|
||
| @Override | ||
| public Offset initialOffset() { | ||
SreeramGarlapati marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| return initialOffset; | ||
| } | ||
|
|
||
| @Override | ||
| public Offset deserializeOffset(String json) { | ||
| return StreamingOffset.fromJson(json); | ||
| } | ||
|
|
||
| @Override | ||
| public void commit(Offset end) { | ||
SreeramGarlapati marked this conversation as resolved.
Show resolved
Hide resolved
SreeramGarlapati marked this conversation as resolved.
Show resolved
Hide resolved
|
||
| } | ||
|
|
||
| @Override | ||
| public void stop() { | ||
| } | ||
|
|
||
| private List<FileScanTask> planFiles(StreamingOffset startOffset, StreamingOffset endOffset) { | ||
| List<FileScanTask> fileScanTasks = Lists.newArrayList(); | ||
| MicroBatch latestMicroBatch = null; | ||
| StreamingOffset batchStartOffset = StreamingOffset.START_OFFSET.equals(startOffset) ? | ||
| new StreamingOffset(SnapshotUtil.oldestSnapshot(table).snapshotId(), 0, false) : | ||
| startOffset; | ||
|
|
||
| do { | ||
| StreamingOffset currentOffset = | ||
| latestMicroBatch != null && latestMicroBatch.lastIndexOfSnapshot() ? | ||
| new StreamingOffset(snapshotAfter(latestMicroBatch.snapshotId()), 0L, false) : | ||
| batchStartOffset; | ||
SreeramGarlapati marked this conversation as resolved.
Show resolved
Hide resolved
|
||
|
|
||
| latestMicroBatch = MicroBatches.from(table.snapshot(currentOffset.snapshotId()), table.io()) | ||
| .caseSensitive(caseSensitive) | ||
| .specsById(table.specs()) | ||
| .generate(currentOffset.position(), Long.MAX_VALUE, currentOffset.shouldScanAllFiles()); | ||
|
|
||
| fileScanTasks.addAll(latestMicroBatch.tasks()); | ||
| } while (latestMicroBatch.snapshotId() != endOffset.snapshotId()); | ||
|
|
||
| return fileScanTasks; | ||
| } | ||
|
|
||
| private long snapshotAfter(long snapshotId) { | ||
| Snapshot snapshotAfter = SnapshotUtil.snapshotAfter(table, snapshotId); | ||
|
|
||
| Preconditions.checkState(snapshotAfter.operation().equals(DataOperations.APPEND), | ||
| "Invalid Snapshot operation: %s, only APPEND is allowed.", snapshotAfter.operation()); | ||
|
|
||
| return snapshotAfter.snapshotId(); | ||
| } | ||
|
|
||
| private static class InitialOffsetStore { | ||
| private final Table table; | ||
| private final FileIO io; | ||
| private final String initialOffsetLocation; | ||
|
|
||
| InitialOffsetStore(Table table, String checkpointLocation) { | ||
| this.table = table; | ||
| this.io = table.io(); | ||
| this.initialOffsetLocation = SLASH.join(checkpointLocation, "offsets/0"); | ||
| } | ||
|
|
||
| public StreamingOffset initialOffset() { | ||
| InputFile inputFile = io.newInputFile(initialOffsetLocation); | ||
| if (inputFile.exists()) { | ||
| return readOffset(inputFile); | ||
| } | ||
|
|
||
| table.refresh(); | ||
| StreamingOffset offset = table.currentSnapshot() == null ? | ||
| StreamingOffset.START_OFFSET : | ||
| new StreamingOffset(SnapshotUtil.oldestSnapshot(table).snapshotId(), 0, false); | ||
|
|
||
| OutputFile outputFile = io.newOutputFile(initialOffsetLocation); | ||
| writeOffset(offset, outputFile); | ||
|
|
||
| return offset; | ||
| } | ||
|
|
||
| private void writeOffset(StreamingOffset offset, OutputFile file) { | ||
| try (OutputStream outputStream = file.create()) { | ||
| BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(outputStream, StandardCharsets.UTF_8)); | ||
| writer.write(offset.json()); | ||
| writer.flush(); | ||
| } catch (IOException ioException) { | ||
| throw new UncheckedIOException( | ||
| String.format("Failed writing offset to: %s", initialOffsetLocation), ioException); | ||
| } | ||
| } | ||
|
|
||
| private StreamingOffset readOffset(InputFile file) { | ||
| try (InputStream in = file.newStream()) { | ||
| return StreamingOffset.fromJson(in); | ||
| } catch (IOException ioException) { | ||
| throw new UncheckedIOException( | ||
| String.format("Failed reading offset from: %s", initialOffsetLocation), ioException); | ||
| } | ||
| } | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
Uh oh!
There was an error while loading. Please reload this page.