[HUDI-2217] Fix no value present in incremental query on MOR (#3340)
This commit is contained in:
@@ -230,6 +230,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assertEquals(1, countsPerCommit.length)
|
||||
assertEquals(firstCommit, countsPerCommit(0).get(0))
|
||||
|
||||
// Test incremental query has no instant in range
|
||||
val emptyIncDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, "001")
|
||||
.load(basePath)
|
||||
assertEquals(0, emptyIncDF.count())
|
||||
|
||||
// Upsert an empty dataFrame
|
||||
val emptyRecords = recordsToStrings(dataGen.generateUpdates("002", 0)).toList
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
|
||||
@@ -201,6 +201,14 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
assertEquals(1, hudiIncDF3.select("_hoodie_commit_time").distinct().count())
|
||||
assertEquals(commit2Time, hudiIncDF3.select("_hoodie_commit_time").head().get(0).toString)
|
||||
|
||||
// Test incremental query has no instant in range
|
||||
val emptyIncDF = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, "001")
|
||||
.load(basePath)
|
||||
assertEquals(0, emptyIncDF.count())
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
|
||||
Reference in New Issue
Block a user