@@ -213,54 +213,50 @@ class TestStreamingSource extends StreamTest {
213
213
}
214
214
215
215
test(" Test mor streaming source with clustering" ) {
216
- Array (" true" , " false" ).foreach(skipCluster => {
217
- withTempDir { inputDir =>
218
- val tablePath = s " ${inputDir.getCanonicalPath}/test_mor_stream_cluster "
219
- val metaClient = HoodieTableMetaClient .newTableBuilder()
220
- .setTableType(MERGE_ON_READ )
221
- .setTableName(getTableName(tablePath))
222
- .setRecordKeyFields(" id" )
223
- .setPreCombineField(" ts" )
224
- .initTable(HadoopFSUtils .getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
216
+ withTempDir { inputDir =>
217
+ val tablePath = s " ${inputDir.getCanonicalPath}/test_mor_stream_cluster "
218
+ val metaClient = HoodieTableMetaClient .newTableBuilder()
219
+ .setTableType(MERGE_ON_READ )
220
+ .setTableName(getTableName(tablePath))
221
+ .setRecordKeyFields(" id" )
222
+ .setPreCombineField(" ts" )
223
+ .initTable(HadoopFSUtils .getStorageConf(spark.sessionState.newHadoopConf()), tablePath)
225
224
226
- addData(tablePath, Seq ((" 1" , " a1" , " 10" , " 000" )))
227
- addData(tablePath, Seq ((" 2" , " a1" , " 11" , " 001" )))
228
- addData(tablePath, Seq ((" 3" , " a1" , " 12" , " 002" )))
229
- addData(tablePath, Seq ((" 4" , " a1" , " 13" , " 003" )), enableInlineCluster = true )
230
- addData(tablePath, Seq ((" 5" , " a1" , " 14" , " 004" )))
225
+ addData(tablePath, Seq ((" 1" , " a1" , " 10" , " 000" )))
226
+ addData(tablePath, Seq ((" 2" , " a1" , " 11" , " 001" )))
227
+ addData(tablePath, Seq ((" 3" , " a1" , " 12" , " 002" )))
228
+ addData(tablePath, Seq ((" 4" , " a1" , " 13" , " 003" )), enableInlineCluster = true )
229
+ addData(tablePath, Seq ((" 5" , " a1" , " 14" , " 004" )))
231
230
232
- val timestamp =
233
- metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants()
234
- .firstInstant().get().getCompletionTime
231
+ val timestamp =
232
+ metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants()
233
+ .firstInstant().get().getCompletionTime
235
234
236
- val df = spark.readStream
237
- .format(" org.apache.hudi" )
238
- .option(START_OFFSET .key(), timestamp)
239
- .option(DataSourceReadOptions .INCREMENTAL_READ_SKIP_CLUSTER .key(), skipCluster)
240
- .load(tablePath)
241
- .select(" id" , " name" , " price" , " ts" )
235
+ val df = spark.readStream
236
+ .format(" org.apache.hudi" )
237
+ .option(START_OFFSET .key(), timestamp)
238
+ .load(tablePath)
239
+ .select(" id" , " name" , " price" , " ts" )
242
240
243
- testStream(df)(
244
- AssertOnQuery { q => q.processAllAvailable(); true },
245
- // Start after the first commit
246
- CheckAnswerRows (Seq (
247
- Row (" 2" , " a1" , " 11" , " 001" ),
248
- Row (" 3" , " a1" , " 12" , " 002" ),
249
- Row (" 4" , " a1" , " 13" , " 003" ),
250
- Row (" 5" , " a1" , " 14" , " 004" )), lastOnly = true , isSorted = false )
251
- )
252
- assertTrue(metaClient.reloadActiveTimeline
253
- .filter(JavaConversions .getPredicate(
254
- e => e.isCompleted && HoodieTimeline .REPLACE_COMMIT_ACTION .equals(e.getAction)))
255
- .countInstants() > 0 )
256
- }
257
- })
241
+ testStream(df)(
242
+ AssertOnQuery { q => q.processAllAvailable(); true },
243
+ // Start after the first commit
244
+ CheckAnswerRows (Seq (
245
+ Row (" 2" , " a1" , " 11" , " 001" ),
246
+ Row (" 3" , " a1" , " 12" , " 002" ),
247
+ Row (" 4" , " a1" , " 13" , " 003" ),
248
+ Row (" 5" , " a1" , " 14" , " 004" )), lastOnly = true , isSorted = false ))
249
+ assertTrue(metaClient.reloadActiveTimeline
250
+ .filter(JavaConversions .getPredicate(
251
+ e => e.isCompleted && HoodieTimeline .REPLACE_COMMIT_ACTION .equals(e.getAction)))
252
+ .countInstants() > 0 )
253
+ }
258
254
}
259
255
260
256
test(" test mor stream source with compaction" ) {
261
257
Array (" true" , " false" ).foreach(skipCompact => {
262
258
withTempDir { inputDir =>
263
- val tablePath = s " ${inputDir.getCanonicalPath}/test_mor_stream "
259
+ val tablePath = s " ${inputDir.getCanonicalPath}/test_mor_stream_ $skipCompact "
264
260
val metaClient = HoodieTableMetaClient .newTableBuilder()
265
261
.setTableType(MERGE_ON_READ )
266
262
.setTableName(getTableName(tablePath))
0 commit comments