diff --git a/kernel-spark/src/main/java/io/delta/kernel/spark/read/IndexedFile.java b/kernel-spark/src/main/java/io/delta/kernel/spark/read/IndexedFile.java
index 49471d196ba..1f7e21b04ad 100644
--- a/kernel-spark/src/main/java/io/delta/kernel/spark/read/IndexedFile.java
+++ b/kernel-spark/src/main/java/io/delta/kernel/spark/read/IndexedFile.java
@@ -16,6 +16,7 @@
package io.delta.kernel.spark.read;
import io.delta.kernel.internal.actions.AddFile;
+import org.apache.spark.sql.delta.sources.AdmittableFile;
/**
* Java version of IndexedFile.scala that uses Kernel's action classes.
@@ -24,7 +25,7 @@
*
*
Indexed: refers to the index in DeltaSourceOffset, assigned by the streaming engine.
*/
-public class IndexedFile {
+public class IndexedFile implements AdmittableFile {
private final long version;
private final long index;
private final AddFile addFile;
@@ -47,6 +48,16 @@ public AddFile getAddFile() {
return addFile;
}
+ @Override
+ public boolean hasFileAction() {
+ return addFile != null;
+ }
+
+ @Override
+ public long getFileSize() {
+ return addFile.getSize();
+ }
+
@Override
public String toString() {
StringBuilder sb = new StringBuilder();
diff --git a/kernel-spark/src/main/java/io/delta/kernel/spark/read/SparkMicroBatchStream.java b/kernel-spark/src/main/java/io/delta/kernel/spark/read/SparkMicroBatchStream.java
index a8b85db29d1..83fbd6d8b54 100644
--- a/kernel-spark/src/main/java/io/delta/kernel/spark/read/SparkMicroBatchStream.java
+++ b/kernel-spark/src/main/java/io/delta/kernel/spark/read/SparkMicroBatchStream.java
@@ -33,6 +33,7 @@
import org.apache.spark.sql.connector.read.streaming.MicroBatchStream;
import org.apache.spark.sql.connector.read.streaming.Offset;
import org.apache.spark.sql.delta.DeltaErrors;
+import org.apache.spark.sql.delta.sources.DeltaSource;
import org.apache.spark.sql.delta.sources.DeltaSourceOffset;
import scala.Option;
@@ -101,6 +102,37 @@ public void stop() {
// getFileChanges //
////////////////////
+ /**
+ * Get file changes with rate limiting applied. Mimics DeltaSource.getFileChangesWithRateLimit.
+ *
+ * @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)
+ * @return An iterator of IndexedFile with rate limiting applied
+ */
+ CloseableIterator getFileChangesWithRateLimit(
+ long fromVersion,
+ long fromIndex,
+ boolean isInitialSnapshot,
+ Option limits) {
+ // TODO(#5319): getFileChangesForCDC if CDC is enabled.
+
+ CloseableIterator changes =
+ getFileChanges(fromVersion, fromIndex, isInitialSnapshot, /*endOffset=*/ Option.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()) {
+ DeltaSource.AdmissionLimits admissionControl = limits.get();
+ changes = changes.takeWhile(admissionControl::admit);
+ }
+
+ // TODO(#5318): Stop at schema change barriers
+ return changes;
+ }
+
/**
* Get file changes between fromVersion/fromIndex and endOffset. This is the Kernel-based
* implementation of DeltaSource.getFileChanges.
diff --git a/kernel-spark/src/test/java/io/delta/kernel/spark/read/SparkMicroBatchStreamTest.java b/kernel-spark/src/test/java/io/delta/kernel/spark/read/SparkMicroBatchStreamTest.java
index 6cc21158141..cd2d5570d35 100644
--- a/kernel-spark/src/test/java/io/delta/kernel/spark/read/SparkMicroBatchStreamTest.java
+++ b/kernel-spark/src/test/java/io/delta/kernel/spark/read/SparkMicroBatchStreamTest.java
@@ -31,6 +31,7 @@
import org.apache.spark.sql.connector.read.streaming.Offset;
import org.apache.spark.sql.delta.DeltaLog;
import org.apache.spark.sql.delta.DeltaOptions;
+import org.apache.spark.sql.delta.sources.DeltaSource;
import org.apache.spark.sql.delta.sources.DeltaSourceOffset;
import org.apache.spark.sql.delta.storage.ClosableIterator;
import org.junit.jupiter.api.BeforeEach;
@@ -138,21 +139,11 @@ public void testGetFileChanges(
// Create 5 versions of data (versions 1-5, version 0 is the CREATE TABLE)
// Insert 100 rows per commit to potentially trigger multiple batches
- for (int i = 0; i < 5; i++) {
- StringBuilder insertValues = new StringBuilder();
- for (int j = 0; j < 100; j++) {
- if (j > 0) insertValues.append(", ");
- int id = i * 100 + j;
- insertValues.append(String.format("(%d, 'User%d')", id, id));
- }
- sql("INSERT INTO %s VALUES %s", testTableName, insertValues.toString());
- }
- SparkMicroBatchStream stream = new SparkMicroBatchStream(testTablePath, new Configuration());
+ insertVersions(testTableName, /* numVersions= */ 5, /* rowsPerVersion= */ 100);
// dsv1 DeltaSource
DeltaLog deltaLog = DeltaLog.forTable(spark, new Path(testTablePath));
- org.apache.spark.sql.delta.sources.DeltaSource deltaSource =
- createDeltaSource(deltaLog, testTablePath);
+ DeltaSource deltaSource = createDeltaSource(deltaLog, testTablePath);
scala.Option scalaEndOffset = scala.Option.empty();
if (endVersion.isPresent()) {
@@ -176,6 +167,7 @@ public void testGetFileChanges(
deltaChanges.close();
// dsv2 SparkMicroBatchStream
+ SparkMicroBatchStream stream = new SparkMicroBatchStream(testTablePath, new Configuration());
Option endOffsetOption = scalaEndOffset;
try (CloseableIterator kernelChanges =
stream.getFileChanges(fromVersion, fromIndex, isInitialSnapshot, endOffsetOption)) {
@@ -237,6 +229,90 @@ private static Stream getFileChangesParameters() {
2L, 50L, notInitialSnapshot, Optional.of(2L), Optional.of(40L), "Empty Range"));
}
+ // ================================================================================================
+ // Tests for getFileChangesWithRateLimit parity between DSv1 and DSv2
+ // ================================================================================================
+
+ /**
+ * Test that verifies parity between DSv1 DeltaSource.getFileChangesWithRateLimit and DSv2
+ * SparkMicroBatchStream.getFileChangesWithRateLimit.
+ */
+ @ParameterizedTest
+ @MethodSource("getFileChangesWithRateLimitParameters")
+ public void testGetFileChangesWithRateLimit(
+ Optional maxFiles,
+ Optional maxBytes,
+ String testDescription,
+ @TempDir File tempDir)
+ throws Exception {
+ String testTablePath = tempDir.getAbsolutePath();
+ String testTableName =
+ "test_rate_limit_" + Math.abs(testDescription.hashCode()) + "_" + System.nanoTime();
+ createEmptyTestTable(testTablePath, testTableName);
+
+ // Create 5 versions with 10 rows each (versions 1-5)
+ insertVersions(testTableName, /* numVersions= */ 5, /* rowsPerVersion= */ 10);
+
+ // dsv1 DeltaSource
+ DeltaLog deltaLog = DeltaLog.forTable(spark, new Path(testTablePath));
+ DeltaSource deltaSource = createDeltaSource(deltaLog, testTablePath);
+ DeltaOptions options = new DeltaOptions(Map$.MODULE$.empty(), spark.sessionState().conf());
+
+ scala.Option