|
32 | 32 | import java.util.stream.Stream; |
33 | 33 | import org.apache.hadoop.conf.Configuration; |
34 | 34 | import org.apache.hadoop.fs.Path; |
| 35 | +import org.apache.spark.sql.catalyst.expressions.Expression; |
35 | 36 | import org.apache.spark.sql.connector.read.streaming.Offset; |
36 | 37 | import org.apache.spark.sql.connector.read.streaming.ReadLimit; |
37 | 38 | import org.apache.spark.sql.delta.DeltaLog; |
38 | 39 | import org.apache.spark.sql.delta.DeltaOptions; |
| 40 | +import org.apache.spark.sql.delta.Snapshot; |
39 | 41 | import org.apache.spark.sql.delta.sources.DeltaSource; |
40 | 42 | import org.apache.spark.sql.delta.sources.DeltaSourceOffset; |
41 | 43 | import org.apache.spark.sql.delta.sources.ReadMaxBytes; |
|
46 | 48 | import org.junit.jupiter.params.provider.Arguments; |
47 | 49 | import org.junit.jupiter.params.provider.MethodSource; |
48 | 50 | import scala.Option; |
| 51 | +import scala.collection.JavaConverters; |
49 | 52 | import scala.collection.immutable.Map$; |
| 53 | +import scala.collection.immutable.Seq; |
50 | 54 |
|
51 | 55 | public class SparkMicroBatchStreamTest extends SparkDsv2TestBase { |
52 | 56 |
|
@@ -100,7 +104,8 @@ public void testInitialOffset_throwsUnsupportedOperationException(@TempDir File |
100 | 104 | SparkMicroBatchStream microBatchStream = createTestStream(tempDir); |
101 | 105 | UnsupportedOperationException exception = |
102 | 106 | assertThrows(UnsupportedOperationException.class, () -> microBatchStream.initialOffset()); |
103 | | - assertEquals("initialOffset is not supported", exception.getMessage()); |
| 107 | + assertEquals( |
| 108 | + "initialOffset with initial snapshot is not supported yet", exception.getMessage()); |
104 | 109 | } |
105 | 110 |
|
106 | 111 | @Test |
@@ -129,6 +134,71 @@ public void testStop_throwsUnsupportedOperationException(@TempDir File tempDir) |
129 | 134 | assertEquals("stop is not supported", exception.getMessage()); |
130 | 135 | } |
131 | 136 |
|
| 137 | + // ================================================================================================ |
| 138 | + // Tests for initialOffset parity between DSv1 and DSv2 |
| 139 | + // ================================================================================================ |
| 140 | + |
| 141 | + @ParameterizedTest |
| 142 | + @MethodSource("initialOffsetParameters") |
| 143 | + public void testInitialOffset_FirstBatchParity( |
| 144 | + String startingVersion, |
| 145 | + ReadLimitConfig limitConfig, |
| 146 | + String testDescription, |
| 147 | + @TempDir File tempDir) |
| 148 | + throws Exception { |
| 149 | + String testTablePath = tempDir.getAbsolutePath(); |
| 150 | + String testTableName = "test_initial_" + System.nanoTime(); |
| 151 | + createEmptyTestTable(testTablePath, testTableName); |
| 152 | + insertVersions( |
| 153 | + testTableName, |
| 154 | + /* numVersions= */ 5, |
| 155 | + /* rowsPerVersion= */ 10, |
| 156 | + /* includeEmptyVersion= */ false); |
| 157 | + |
| 158 | + DeltaLog deltaLog = DeltaLog.forTable(spark, new Path(testTablePath)); |
| 159 | + ReadLimit readLimit = limitConfig.toReadLimit(); |
| 160 | + DeltaOptions options; |
| 161 | + if (startingVersion == null) { |
| 162 | + options = new DeltaOptions(Map$.MODULE$.empty(), spark.sessionState().conf()); |
| 163 | + } else { |
| 164 | + scala.collection.immutable.Map<String, String> scalaMap = |
| 165 | + Map$.MODULE$.<String, String>empty().updated("startingVersion", startingVersion); |
| 166 | + options = new DeltaOptions(scalaMap, spark.sessionState().conf()); |
| 167 | + } |
| 168 | + |
| 169 | + // DSv1 |
| 170 | + DeltaSource deltaSource = createDeltaSource(deltaLog, testTablePath, options); |
| 171 | + // DSv1 sources don't have an initialOffset() method. |
| 172 | + // Batch 0 is called with startOffset=null. |
| 173 | + Offset dsv1Offset = deltaSource.latestOffset(/* startOffset= */ null, readLimit); |
| 174 | + |
| 175 | + // DSv2 |
| 176 | + Configuration hadoopConf = new Configuration(); |
| 177 | + PathBasedSnapshotManager snapshotManager = |
| 178 | + new PathBasedSnapshotManager(testTablePath, hadoopConf); |
| 179 | + SparkMicroBatchStream stream = |
| 180 | + new SparkMicroBatchStream(snapshotManager, hadoopConf, spark, options); |
| 181 | + Offset initialOffset = stream.initialOffset(); |
| 182 | + Offset dsv2Offset = stream.latestOffset(initialOffset, readLimit); |
| 183 | + |
| 184 | + compareOffsets(dsv1Offset, dsv2Offset, testDescription); |
| 185 | + } |
| 186 | + |
| 187 | + /** Provides test parameters for the initialOffset parity test. */ |
| 188 | + private static Stream<Arguments> initialOffsetParameters() { |
| 189 | + return Stream.of( |
| 190 | + Arguments.of("0", ReadLimitConfig.noLimit(), "NoLimit1"), |
| 191 | + Arguments.of("1", ReadLimitConfig.noLimit(), "NoLimit2"), |
| 192 | + Arguments.of("3", ReadLimitConfig.noLimit(), "NoLimit3"), |
| 193 | + Arguments.of("latest", ReadLimitConfig.noLimit(), "LatestNoLimit"), |
| 194 | + Arguments.of("latest", ReadLimitConfig.maxFiles(1000), "LatestMaxFiles"), |
| 195 | + Arguments.of("latest", ReadLimitConfig.maxBytes(1000), "LatestMaxBytes"), |
| 196 | + Arguments.of("0", ReadLimitConfig.maxFiles(5), "MaxFiles1"), |
| 197 | + Arguments.of("1", ReadLimitConfig.maxFiles(10), "MaxFiles2"), |
| 198 | + Arguments.of("0", ReadLimitConfig.maxBytes(1000), "MaxBytes1"), |
| 199 | + Arguments.of("1", ReadLimitConfig.maxBytes(2000), "MaxBytes2")); |
| 200 | + } |
| 201 | + |
132 | 202 | // ================================================================================================ |
133 | 203 | // Tests for getFileChanges parity between DSv1 and DSv2 |
134 | 204 | // ================================================================================================ |
@@ -1095,26 +1165,6 @@ private Optional<DeltaSource.AdmissionLimits> createAdmissionLimits( |
1095 | 1165 | return Optional.of(new DeltaSource.AdmissionLimits(options, scalaMaxFiles, scalaMaxBytes)); |
1096 | 1166 | } |
1097 | 1167 |
|
1098 | | - /** Helper method to create a DeltaSource instance for testing. */ |
1099 | | - private DeltaSource createDeltaSource(DeltaLog deltaLog, String tablePath) { |
1100 | | - DeltaOptions options = new DeltaOptions(Map$.MODULE$.empty(), spark.sessionState().conf()); |
1101 | | - scala.collection.immutable.Seq<org.apache.spark.sql.catalyst.expressions.Expression> emptySeq = |
1102 | | - scala.collection.JavaConverters.asScalaBuffer( |
1103 | | - new java.util.ArrayList<org.apache.spark.sql.catalyst.expressions.Expression>()) |
1104 | | - .toList(); |
1105 | | - org.apache.spark.sql.delta.Snapshot snapshot = |
1106 | | - deltaLog.update(false, scala.Option.empty(), scala.Option.empty()); |
1107 | | - return new DeltaSource( |
1108 | | - spark, |
1109 | | - deltaLog, |
1110 | | - /* catalogTableOpt= */ scala.Option.empty(), |
1111 | | - options, |
1112 | | - /* snapshotAtSourceInit= */ snapshot, |
1113 | | - /* metadataPath= */ tablePath + "/_checkpoint", |
1114 | | - /* metadataTrackingLog= */ scala.Option.empty(), |
1115 | | - /* filters= */ emptySeq); |
1116 | | - } |
1117 | | - |
1118 | 1168 | /** Helper method to format a DSv1 IndexedFile for debugging. */ |
1119 | 1169 | private String formatIndexedFile(org.apache.spark.sql.delta.sources.IndexedFile file) { |
1120 | 1170 | return String.format( |
@@ -1173,4 +1223,23 @@ private void compareOffsetSequence( |
1173 | 1223 | String.format("%s (iteration %d)", testDescription, i)); |
1174 | 1224 | } |
1175 | 1225 | } |
| 1226 | + |
| 1227 | + private DeltaSource createDeltaSource(DeltaLog deltaLog, String tablePath) { |
| 1228 | + DeltaOptions options = new DeltaOptions(Map$.MODULE$.empty(), spark.sessionState().conf()); |
| 1229 | + return createDeltaSource(deltaLog, tablePath, options); |
| 1230 | + } |
| 1231 | + |
| 1232 | + private DeltaSource createDeltaSource(DeltaLog deltaLog, String tablePath, DeltaOptions options) { |
| 1233 | + Seq<Expression> emptySeq = JavaConverters.asScalaBuffer(new ArrayList<Expression>()).toList(); |
| 1234 | + Snapshot snapshot = deltaLog.update(false, Option.empty(), Option.empty()); |
| 1235 | + return new DeltaSource( |
| 1236 | + spark, |
| 1237 | + deltaLog, |
| 1238 | + /* catalogTableOpt= */ Option.empty(), |
| 1239 | + options, |
| 1240 | + /* snapshotAtSourceInit= */ snapshot, |
| 1241 | + /* metadataPath= */ tablePath + "/_checkpoint", |
| 1242 | + /* metadataTrackingLog= */ Option.empty(), |
| 1243 | + /* filters= */ emptySeq); |
| 1244 | + } |
1176 | 1245 | } |
0 commit comments