-
Notifications
You must be signed in to change notification settings - Fork 2k
[kernel-spark] Implement latestOffset() with rate limiting for dsv2 streaming #5409
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
Changes from all commits
e9e8665
bdacbcf
b0656dd
085fa9f
720aa62
9f3fe88
639ee93
eca74b1
f214803
d49a65c
8e6d107
7c134ad
7064800
1179443
7d9d642
1718356
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,10 +22,12 @@ | |
| import io.delta.kernel.engine.Engine; | ||
| import io.delta.kernel.exceptions.UnsupportedTableFeatureException; | ||
| import io.delta.kernel.internal.DeltaLogActionUtils.DeltaAction; | ||
| import io.delta.kernel.internal.SnapshotImpl; | ||
| import io.delta.kernel.internal.actions.AddFile; | ||
| import io.delta.kernel.internal.actions.RemoveFile; | ||
| import io.delta.kernel.internal.util.Utils; | ||
| import io.delta.kernel.spark.snapshot.DeltaSnapshotManager; | ||
| import io.delta.kernel.spark.utils.ScalaUtils; | ||
| import io.delta.kernel.spark.utils.StreamingHelper; | ||
| import io.delta.kernel.utils.CloseableIterator; | ||
| import java.io.IOException; | ||
|
|
@@ -34,8 +36,7 @@ | |
| 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.connector.read.streaming.*; | ||
| import org.apache.spark.sql.delta.DeltaErrors; | ||
| import org.apache.spark.sql.delta.DeltaOptions; | ||
| import org.apache.spark.sql.delta.DeltaStartingVersion; | ||
|
|
@@ -46,9 +47,8 @@ | |
| import org.apache.spark.sql.delta.sources.DeltaSourceOffset; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
| import scala.Option; | ||
|
|
||
| public class SparkMicroBatchStream implements MicroBatchStream { | ||
| public class SparkMicroBatchStream implements MicroBatchStream, SupportsAdmissionControl { | ||
|
|
||
| private static final Logger logger = LoggerFactory.getLogger(SparkMicroBatchStream.class); | ||
|
|
||
|
|
@@ -59,6 +59,8 @@ public class SparkMicroBatchStream implements MicroBatchStream { | |
| private final Engine engine; | ||
| private final DeltaSnapshotManager snapshotManager; | ||
| private final DeltaOptions options; | ||
| private final String tableId; | ||
| private final boolean shouldValidateOffsets; | ||
| private final SparkSession spark; | ||
|
|
||
| public SparkMicroBatchStream(DeltaSnapshotManager snapshotManager, Configuration hadoopConf) { | ||
|
|
@@ -80,6 +82,13 @@ public SparkMicroBatchStream( | |
| this.snapshotManager = snapshotManager; | ||
| this.engine = DefaultEngine.create(hadoopConf); | ||
| this.options = options; | ||
|
|
||
| // Initialize snapshot at source init to get table ID, similar to DeltaSource.scala | ||
| Snapshot snapshotAtSourceInit = snapshotManager.loadLatestSnapshot(); | ||
| this.tableId = ((SnapshotImpl) snapshotAtSourceInit).getMetadata().getId(); | ||
|
|
||
| this.shouldValidateOffsets = | ||
| (Boolean) spark.sessionState().conf().getConf(DeltaSQLConf.STREAMING_OFFSET_VALIDATION()); | ||
| } | ||
|
|
||
| //////////// | ||
|
|
@@ -88,19 +97,91 @@ public SparkMicroBatchStream( | |
|
|
||
| @Override | ||
| public Offset initialOffset() { | ||
| // TODO(#5318): Implement initialOffset | ||
| throw new UnsupportedOperationException("initialOffset is not supported"); | ||
| } | ||
|
|
||
| @Override | ||
| public Offset latestOffset() { | ||
| throw new UnsupportedOperationException("latestOffset is not supported"); | ||
| throw new IllegalStateException( | ||
| "latestOffset() should not be called - use latestOffset(Offset, ReadLimit) instead"); | ||
| } | ||
|
|
||
| /** | ||
| * Get the latest offset with rate limiting (SupportsAdmissionControl). | ||
| * | ||
| * @param startOffset The starting offset (can be null if initialOffset() returned null) | ||
| * @param limit The read limit for rate limiting | ||
| * @return The latest offset, or null if no data is available to read. | ||
| */ | ||
| @Override | ||
| public Offset latestOffset(Offset startOffset, ReadLimit limit) { | ||
| // For the first batch, initialOffset() should be called before latestOffset(). | ||
| // if startOffset is null: no data is available to read. | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Will there by a case that startOffset is set to null
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Spark will call initialOffset first to obtain this startOffset for batch 0 -- so it could be null when initialOffset() returns null (i.e. table has no data). |
||
| if (startOffset == null) { | ||
| return null; | ||
| } | ||
| // TODO(#5318): init trigger available now support | ||
|
|
||
| DeltaSourceOffset deltaStartOffset = DeltaSourceOffset.apply(tableId, startOffset); | ||
| Optional<DeltaSource.AdmissionLimits> limits = | ||
| ScalaUtils.toJavaOptional(DeltaSource.AdmissionLimits$.MODULE$.apply(options, limit)); | ||
| Optional<DeltaSourceOffset> endOffset = | ||
| getNextOffsetFromPreviousOffset(deltaStartOffset, limits); | ||
|
|
||
| if (shouldValidateOffsets && endOffset.isPresent()) { | ||
| DeltaSourceOffset.validateOffsets(deltaStartOffset, endOffset.get()); | ||
| } | ||
|
|
||
| // endOffset is null: no data is available to read for this batch. | ||
| return endOffset.orElse(null); | ||
| } | ||
|
|
||
| @Override | ||
| public Offset deserializeOffset(String json) { | ||
| throw new UnsupportedOperationException("deserializeOffset is not supported"); | ||
| } | ||
|
|
||
| @Override | ||
| public ReadLimit getDefaultReadLimit() { | ||
| return DeltaSource.AdmissionLimits$.MODULE$.toReadLimit(options); | ||
| } | ||
|
|
||
| /** | ||
| * Return the next offset when previous offset exists. Mimics | ||
| * DeltaSource.getNextOffsetFromPreviousOffset. | ||
| * | ||
| * @param previousOffset The previous offset | ||
| * @param limits Rate limits for this batch (Optional.empty() for no limits) | ||
| * @return The next offset, or the previous offset if no new data is available | ||
| */ | ||
| private Optional<DeltaSourceOffset> getNextOffsetFromPreviousOffset( | ||
| DeltaSourceOffset previousOffset, Optional<DeltaSource.AdmissionLimits> limits) { | ||
| // TODO(#5319): Special handling for schema tracking. | ||
|
|
||
| CloseableIterator<IndexedFile> changes = | ||
| getFileChangesWithRateLimit( | ||
| previousOffset.reservoirVersion(), | ||
| previousOffset.index(), | ||
| previousOffset.isInitialSnapshot(), | ||
| limits); | ||
|
|
||
| Optional<IndexedFile> lastFileChange = Utils.iteratorLast(changes); | ||
|
|
||
| if (!lastFileChange.isPresent()) { | ||
| return Optional.of(previousOffset); | ||
| } | ||
| // TODO(#5318): Check read-incompatible schema changes during stream start | ||
| IndexedFile lastFile = lastFileChange.get(); | ||
| return Optional.of( | ||
| DeltaSource.buildOffsetFromIndexedFile( | ||
|
Collaborator
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Can we update the signature for
Collaborator
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done. |
||
| tableId, | ||
| lastFile.getVersion(), | ||
| lastFile.getIndex(), | ||
| previousOffset.reservoirVersion(), | ||
| previousOffset.isInitialSnapshot())); | ||
| } | ||
|
|
||
| //////////// | ||
| /// data /// | ||
| //////////// | ||
|
|
@@ -214,23 +295,24 @@ private static boolean validateProtocolAt( | |
| * @param fromVersion The starting version (exclusive with fromIndex) | ||
| * @param fromIndex The starting index within fromVersion (exclusive) | ||
| * @param isInitialSnapshot Whether this is the initial snapshot | ||
| * @param limits Rate limits to apply (Option.empty for no limits) | ||
| * @param limits Rate limits to apply (Optional.empty() for no limits) | ||
| * @return An iterator of IndexedFile with rate limiting applied | ||
| */ | ||
| CloseableIterator<IndexedFile> getFileChangesWithRateLimit( | ||
| long fromVersion, | ||
| long fromIndex, | ||
| boolean isInitialSnapshot, | ||
| Option<DeltaSource.AdmissionLimits> limits) { | ||
| Optional<DeltaSource.AdmissionLimits> limits) { | ||
| // TODO(#5319): getFileChangesForCDC if CDC is enabled. | ||
|
|
||
| CloseableIterator<IndexedFile> changes = | ||
| getFileChanges(fromVersion, fromIndex, isInitialSnapshot, /*endOffset=*/ Option.empty()); | ||
| getFileChanges( | ||
| fromVersion, fromIndex, isInitialSnapshot, /* endOffset= */ Optional.empty()); | ||
|
|
||
| // Take each change until we've seen the configured number of addFiles. Some changes don't | ||
| // represent file additions; we retain them for offset tracking, but they don't count toward | ||
| // the maxFilesPerTrigger conf. | ||
| if (limits.isDefined()) { | ||
| if (limits.isPresent()) { | ||
| DeltaSource.AdmissionLimits admissionLimits = limits.get(); | ||
| changes = changes.takeWhile(admissionLimits::admit); | ||
| } | ||
|
|
@@ -255,7 +337,7 @@ CloseableIterator<IndexedFile> getFileChanges( | |
| long fromVersion, | ||
| long fromIndex, | ||
| boolean isInitialSnapshot, | ||
| Option<DeltaSourceOffset> endOffset) { | ||
| Optional<DeltaSourceOffset> endOffset) { | ||
|
|
||
| CloseableIterator<IndexedFile> result; | ||
|
|
||
|
|
@@ -274,7 +356,7 @@ CloseableIterator<IndexedFile> getFileChanges( | |
| || (file.getVersion() == fromVersion && file.getIndex() > fromIndex)); | ||
|
|
||
| // Check end boundary (inclusive) | ||
| if (endOffset.isDefined()) { | ||
| if (endOffset.isPresent()) { | ||
| DeltaSourceOffset bound = endOffset.get(); | ||
| result = | ||
| result.takeWhile( | ||
|
|
@@ -289,11 +371,20 @@ CloseableIterator<IndexedFile> getFileChanges( | |
|
|
||
| // TODO(#5318): implement lazy loading (one batch at a time). | ||
| private CloseableIterator<IndexedFile> filterDeltaLogs( | ||
| long startVersion, Option<DeltaSourceOffset> endOffset) { | ||
| long startVersion, Optional<DeltaSourceOffset> endOffset) { | ||
| List<IndexedFile> allIndexedFiles = new ArrayList<>(); | ||
| Optional<Long> endVersionOpt = | ||
| endOffset.isDefined() ? Optional.of(endOffset.get().reservoirVersion()) : Optional.empty(); | ||
| CommitRange commitRange = snapshotManager.getTableChanges(engine, startVersion, endVersionOpt); | ||
| endOffset.isPresent() ? Optional.of(endOffset.get().reservoirVersion()) : Optional.empty(); | ||
|
|
||
| CommitRange commitRange; | ||
| try { | ||
| commitRange = snapshotManager.getTableChanges(engine, startVersion, endVersionOpt); | ||
| } catch (io.delta.kernel.exceptions.CommitRangeNotFoundException e) { | ||
| // If the requested version range doesn't exist (e.g., we're asking for version 6 when | ||
| // the table only has versions 0-5). | ||
| return Utils.toCloseableIterator(allIndexedFiles.iterator()); | ||
| } | ||
|
|
||
| // Required by kernel: perform protocol validation by creating a snapshot at startVersion. | ||
| Snapshot startSnapshot = snapshotManager.loadSnapshotAt(startVersion); | ||
| String tablePath = startSnapshot.getPath(); | ||
|
|
@@ -374,9 +465,12 @@ private void flushVersion( | |
| * @throws RuntimeException if the commit is invalid. | ||
| */ | ||
| private void validateCommit( | ||
| ColumnarBatch batch, long version, String tablePath, Option<DeltaSourceOffset> endOffsetOpt) { | ||
| ColumnarBatch batch, | ||
| long version, | ||
| String tablePath, | ||
| Optional<DeltaSourceOffset> endOffsetOpt) { | ||
| // If endOffset is at the beginning of this version, exit early. | ||
| if (endOffsetOpt.isDefined()) { | ||
| if (endOffsetOpt.isPresent()) { | ||
| DeltaSourceOffset endOffset = endOffsetOpt.get(); | ||
| if (endOffset.reservoirVersion() == version | ||
| && endOffset.index() == DeltaSourceOffset.BASE_INDEX()) { | ||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.