[HUDI-920] Support Incremental query for MOR table (#1938)
This commit is contained in:
@@ -19,7 +19,7 @@ package org.apache.hudi.functional
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.{HoodieCompactionConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
import org.apache.hudi.testutils.HoodieClientTestBase
|
||||
@@ -29,6 +29,7 @@ import org.apache.spark.sql.functions._
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
|
||||
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
|
||||
/**
|
||||
@@ -157,6 +158,39 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
assertTrue(commit2Time > commit1Time)
|
||||
assertEquals(100, hudiSnapshotDF2.join(hudiSnapshotDF1, Seq("_hoodie_record_key"), "left").count())
|
||||
|
||||
// incremental view
|
||||
// base file only
|
||||
val hudiIncDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit1Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF1.count())
|
||||
assertEquals(1, hudiIncDF1.select("_hoodie_commit_time").distinct().count())
|
||||
assertEquals(commit1Time, hudiIncDF1.select("_hoodie_commit_time").head().get(0).toString)
|
||||
hudiIncDF1.show(1)
|
||||
// log file only
|
||||
val hudiIncDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit1Time)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF2.count())
|
||||
assertEquals(1, hudiIncDF2.select("_hoodie_commit_time").distinct().count())
|
||||
assertEquals(commit2Time, hudiIncDF2.select("_hoodie_commit_time").head().get(0).toString)
|
||||
hudiIncDF2.show(1)
|
||||
|
||||
// base file + log file
|
||||
val hudiIncDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF3.count())
|
||||
// log file being load
|
||||
assertEquals(1, hudiIncDF3.select("_hoodie_commit_time").distinct().count())
|
||||
assertEquals(commit2Time, hudiIncDF3.select("_hoodie_commit_time").head().get(0).toString)
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
@@ -193,6 +227,22 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
assertEquals(50,
|
||||
hudiSnapshotDF3.join(hudiSnapshotDF2, Seq("_hoodie_record_key", "_hoodie_commit_time"), "inner").count())
|
||||
|
||||
// incremental query from commit2Time
|
||||
val hudiIncDF4 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(50, hudiIncDF4.count())
|
||||
|
||||
// skip merge incremental view
|
||||
// including commit 2 and commit 3
|
||||
val hudiIncDF4SkipMerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(200, hudiIncDF4SkipMerge.count())
|
||||
|
||||
// Fourth Operation:
|
||||
// Insert records to a new partition. Produced a new parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition and parquet from the new partition.
|
||||
@@ -213,21 +263,51 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
assertEquals(100,
|
||||
hudiSnapshotDF1.join(hudiSnapshotDF4, Seq("_hoodie_record_key"), "inner").count())
|
||||
|
||||
// Incremental query, 50 from log file, 100 from base file of the new partition.
|
||||
val hudiIncDF5 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(150, hudiIncDF5.count())
|
||||
|
||||
// Fifth Operation:
|
||||
// Upsert records to the new partition. Produced a newer version of parquet file.
|
||||
// SNAPSHOT view should read the latest log files from the default partition
|
||||
// and the latest parquet from the new partition.
|
||||
val records5 = recordsToStrings(newDataGen.generateUpdates("005", 100)).toList
|
||||
val records5 = recordsToStrings(newDataGen.generateUniqueUpdates("005", 50)).toList
|
||||
val inputDF5: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records5, 2))
|
||||
inputDF5.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option("hoodie.compact.inline", "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val commit5Time = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF5 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotDF5.count())
|
||||
|
||||
// Sixth Operation:
|
||||
// Insert 2 records and trigger compaction.
|
||||
val records6 = recordsToStrings(newDataGen.generateInserts("006", 2)).toList
|
||||
val inputDF6: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records6, 2))
|
||||
inputDF6.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val commit6Time = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF6 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/2020/01/10/*")
|
||||
assertEquals(102, hudiSnapshotDF6.count())
|
||||
val hudiIncDF6 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit5Time)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, commit6Time)
|
||||
.load(basePath)
|
||||
// compaction updated 150 rows + inserted 2 new row
|
||||
assertEquals(152, hudiIncDF6.count())
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -276,6 +356,13 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2Unmerge.count())
|
||||
|
||||
// incremental query, read 50 delete records from log file and get 0 count.
|
||||
val hudiIncDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(0, hudiIncDF1.count())
|
||||
|
||||
// Third Operation:
|
||||
// Upsert 50 delete records to delete the reset
|
||||
// Snopshot view should read 0 record
|
||||
@@ -308,6 +395,8 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
@@ -329,34 +418,43 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
val hudiIncDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.load(basePath)
|
||||
val hudiIncDF1Skipmerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.load(basePath)
|
||||
val hudiIncDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commit1Time)
|
||||
.load(basePath)
|
||||
|
||||
// filter first commit and only read log records
|
||||
assertEquals(50, hudiSnapshotDF2.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history")
|
||||
.filter(col("_hoodie_commit_time") > commit1Time).count())
|
||||
assertEquals(50, hudiIncDF1.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history")
|
||||
.filter(col("_hoodie_commit_time") > commit1Time).count())
|
||||
assertEquals(50, hudiIncDF2
|
||||
.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").count())
|
||||
assertEquals(150, hudiIncDF1Skipmerge
|
||||
.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").count())
|
||||
|
||||
// select nested columns with order different from the actual schema
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
hudiSnapshotDF2
|
||||
.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
|
||||
// Correctly loading type
|
||||
val sampleRow = hudiSnapshotDF2
|
||||
.select("begin_lat", "current_date", "fare.currency", "tip_history", "nation")
|
||||
.orderBy(desc("_hoodie_commit_time"))
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getString(2), sampleRow.get(2))
|
||||
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
|
||||
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
|
||||
verifySchemaAndTypes(hudiSnapshotDF1)
|
||||
verifySchemaAndTypes(hudiSnapshotDF2)
|
||||
verifySchemaAndTypes(hudiIncDF1)
|
||||
verifySchemaAndTypes(hudiIncDF2)
|
||||
verifySchemaAndTypes(hudiIncDF1Skipmerge)
|
||||
|
||||
// make sure show() work
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
verifyShow(hudiSnapshotDF1)
|
||||
verifyShow(hudiSnapshotDF2)
|
||||
verifyShow(hudiIncDF1)
|
||||
verifyShow(hudiIncDF2)
|
||||
verifyShow(hudiIncDF1Skipmerge)
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -404,4 +502,25 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
hudiSnapshotDF1.show(1)
|
||||
hudiSnapshotDF2.show(1)
|
||||
}
|
||||
|
||||
def verifySchemaAndTypes(df: DataFrame): Unit = {
|
||||
assertEquals("amount,currency,tip_history,_hoodie_commit_seqno",
|
||||
df.select("fare.amount", "fare.currency", "tip_history", "_hoodie_commit_seqno")
|
||||
.orderBy(desc("_hoodie_commit_seqno"))
|
||||
.columns.mkString(","))
|
||||
val sampleRow = df
|
||||
.select("begin_lat", "current_date", "fare.currency", "tip_history", "nation")
|
||||
.orderBy(desc("_hoodie_commit_time"))
|
||||
.head()
|
||||
assertEquals(sampleRow.getDouble(0), sampleRow.get(0))
|
||||
assertEquals(sampleRow.getLong(1), sampleRow.get(1))
|
||||
assertEquals(sampleRow.getString(2), sampleRow.get(2))
|
||||
assertEquals(sampleRow.getSeq(3), sampleRow.get(3))
|
||||
assertEquals(sampleRow.getStruct(4), sampleRow.get(4))
|
||||
}
|
||||
|
||||
def verifyShow(df: DataFrame): Unit = {
|
||||
df.show(1)
|
||||
df.select("_hoodie_commit_seqno", "fare.amount", "fare.currency", "tip_history").show(1)
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user