[HUDI-509] Renaming code in sync with cWiki restructuring (#1212)
- Storage Type replaced with Table Type (remaining instances) - View types replaced with query types; - ReadOptimized view referred as Snapshot Query - TableFileSystemView sub interfaces renamed to BaseFileOnly and Slice Views - HoodieDataFile renamed to HoodieBaseFile - Hive Sync tool will register RO tables for MOR with a `_ro` suffix - Datasource/Deltastreamer options renamed accordingly - Support fallback to old config values as well, so migration is painless - Config for controlling _ro suffix addition - Renaming DataFile to BaseFile across DTOs, HoodieFileSlice and AbstractTableFileSystemView
This commit is contained in:
@@ -132,7 +132,7 @@ public class HoodieJavaApp {
|
||||
// full list in HoodieWriteConfig & its package
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
// Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
// insert
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
// This is the record key
|
||||
@@ -165,7 +165,7 @@ public class HoodieJavaApp {
|
||||
Dataset<Row> inputDF2 = spark.read().json(jssc.parallelize(records2, 2));
|
||||
writer = inputDF2.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
@@ -188,7 +188,7 @@ public class HoodieJavaApp {
|
||||
Dataset<Row> inputDF3 = spark.read().json(jssc.parallelize(deletes, 2));
|
||||
writer = inputDF3.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), "delete")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
@@ -206,11 +206,11 @@ public class HoodieJavaApp {
|
||||
/**
|
||||
* Read & do some queries
|
||||
*/
|
||||
Dataset<Row> hoodieROViewDF = spark.read().format("org.apache.hudi")
|
||||
Dataset<Row> snapshotQueryDF = spark.read().format("org.apache.hudi")
|
||||
// pass any path glob, can include hoodie & non-hoodie
|
||||
// datasets
|
||||
.load(tablePath + (nonPartitionedTable ? "/*" : "/*/*/*/*"));
|
||||
hoodieROViewDF.registerTempTable("hoodie_ro");
|
||||
snapshotQueryDF.registerTempTable("hoodie_ro");
|
||||
spark.sql("describe hoodie_ro").show();
|
||||
// all trips whose fare amount was greater than 2.
|
||||
spark.sql("select fare.amount, begin_lon, begin_lat, timestamp from hoodie_ro where fare.amount > 2.0").show();
|
||||
@@ -219,15 +219,15 @@ public class HoodieJavaApp {
|
||||
/**
|
||||
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
|
||||
*/
|
||||
Dataset<Row> hoodieIncViewDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
|
||||
Dataset<Row> incQueryDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
.load(tablePath);
|
||||
|
||||
LOG.info("You will only see records from : " + commitInstantTime2);
|
||||
hoodieIncViewDF.groupBy(hoodieIncViewDF.col("_hoodie_commit_time")).count().show();
|
||||
incQueryDF.groupBy(incQueryDF.col("_hoodie_commit_time")).count().show();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -203,7 +203,7 @@ public class HoodieJavaStreamingApp {
|
||||
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
|
||||
*/
|
||||
Dataset<Row> hoodieIncViewDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY(), DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
@@ -224,7 +224,7 @@ public class HoodieJavaStreamingApp {
|
||||
|
||||
DataStreamWriter<Row> writer = streamingInput.writeStream().format("org.apache.hudi")
|
||||
.option("hoodie.insert.shuffle.parallelism", "2").option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
|
||||
@@ -116,7 +116,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, basePath, "000").get(0);
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.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, firstCommit)
|
||||
.load(basePath);
|
||||
@@ -127,7 +127,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath);
|
||||
|
||||
@@ -145,7 +145,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
.options(commonOpts)
|
||||
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -193,7 +193,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
@@ -263,7 +263,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
// we have 2 commits, try pulling the first commit (which is not the latest)
|
||||
val firstCommit = HoodieDataSourceHelpers.listCommitsSince(fs, destPath, "000").get(0)
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.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, firstCommit)
|
||||
.load(destPath)
|
||||
@@ -275,7 +275,7 @@ class TestDataSource extends AssertionsForJUnit {
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.VIEW_TYPE_OPT_KEY, DataSourceReadOptions.VIEW_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
|
||||
@@ -29,13 +29,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val originals = HoodieSparkSqlWriter.parametersWithWriteDefaults(Map.empty)
|
||||
val rhsKey = "hoodie.right.hand.side.key"
|
||||
val rhsVal = "hoodie.right.hand.side.val"
|
||||
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, STORAGE_TYPE_OPT_KEY -> MOR_STORAGE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modifier = Map(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modified = HoodieSparkSqlWriter.parametersWithWriteDefaults(modifier)
|
||||
val matcher = (k: String, v: String) => modified(k) should be(v)
|
||||
|
||||
originals foreach {
|
||||
case (OPERATION_OPT_KEY, _) => matcher(OPERATION_OPT_KEY, INSERT_OPERATION_OPT_VAL)
|
||||
case (STORAGE_TYPE_OPT_KEY, _) => matcher(STORAGE_TYPE_OPT_KEY, MOR_STORAGE_TYPE_OPT_VAL)
|
||||
case (TABLE_TYPE_OPT_KEY, _) => matcher(TABLE_TYPE_OPT_KEY, MOR_TABLE_TYPE_OPT_VAL)
|
||||
case (`rhsKey`, _) => matcher(rhsKey, rhsVal)
|
||||
case (k, v) => matcher(k, v)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user