[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:
@@ -138,8 +138,8 @@ public class DataSourceUtils {
|
||||
String tblName, Map<String, String> parameters) {
|
||||
|
||||
// inline compaction is on by default for MOR
|
||||
boolean inlineCompact = parameters.get(DataSourceWriteOptions.STORAGE_TYPE_OPT_KEY())
|
||||
.equals(DataSourceWriteOptions.MOR_STORAGE_TYPE_OPT_VAL());
|
||||
boolean inlineCompact = parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY())
|
||||
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
|
||||
|
||||
// insert/bulk-insert combining to be true, if filtering for duplicates
|
||||
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
|
||||
|
||||
@@ -21,6 +21,7 @@ import org.apache.hudi.common.model.HoodieTableType
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
|
||||
import org.apache.hudi.keygen.SimpleKeyGenerator
|
||||
import org.apache.log4j.LogManager
|
||||
|
||||
/**
|
||||
* List of options that can be passed to the Hoodie datasource,
|
||||
@@ -31,20 +32,49 @@ import org.apache.hudi.keygen.SimpleKeyGenerator
|
||||
* Options supported for reading hoodie tables.
|
||||
*/
|
||||
object DataSourceReadOptions {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[DefaultSource])
|
||||
|
||||
/**
|
||||
* Whether data needs to be read, in
|
||||
* incremental mode (new data since an instantTime)
|
||||
* (or) Read Optimized mode (obtain latest view, based on columnar data)
|
||||
* (or) Real time mode (obtain latest view, based on row & columnar data)
|
||||
*
|
||||
* Default: READ_OPTIMIZED
|
||||
* 1) Snapshot mode (obtain latest view, based on row & columnar data)
|
||||
* 2) incremental mode (new data since an instantTime)
|
||||
* 3) Read Optimized mode (obtain latest view, based on columnar data)
|
||||
*
|
||||
* Default: snapshot
|
||||
*/
|
||||
val QUERY_TYPE_OPT_KEY = "hoodie.datasource.query.type"
|
||||
val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
|
||||
val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
|
||||
val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
|
||||
val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
|
||||
|
||||
@Deprecated
|
||||
val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type"
|
||||
@Deprecated
|
||||
val VIEW_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
|
||||
@Deprecated
|
||||
val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental"
|
||||
@Deprecated
|
||||
val VIEW_TYPE_REALTIME_OPT_VAL = "realtime"
|
||||
@Deprecated
|
||||
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
|
||||
val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = ""
|
||||
|
||||
/**
|
||||
* This eases migration from old configs to new configs.
|
||||
*/
|
||||
def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
|
||||
val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
|
||||
VIEW_TYPE_INCREMENTAL_OPT_VAL -> QUERY_TYPE_INCREMENTAL_OPT_VAL,
|
||||
VIEW_TYPE_REALTIME_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
if (optParams.contains(VIEW_TYPE_OPT_KEY) && !optParams.contains(QUERY_TYPE_OPT_KEY)) {
|
||||
log.warn(VIEW_TYPE_OPT_KEY + " is deprecated and will be removed in a later release. Please use " + QUERY_TYPE_OPT_KEY)
|
||||
optParams ++ Map(QUERY_TYPE_OPT_KEY -> translation(optParams(VIEW_TYPE_OPT_KEY)))
|
||||
} else {
|
||||
optParams
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Instant time to start incrementally pulling data from. The instanttime here need not
|
||||
@@ -72,14 +102,18 @@ object DataSourceReadOptions {
|
||||
* This option allows setting filters directly on Hoodie Source
|
||||
*/
|
||||
val PUSH_DOWN_INCR_FILTERS_OPT_KEY = "hoodie.datasource.read.incr.filters"
|
||||
val DEFAULTPUSH_DOWN_FILTERS_OPT_VAL = ""
|
||||
}
|
||||
|
||||
/**
|
||||
* Options supported for writing hoodie tables.
|
||||
*/
|
||||
object DataSourceWriteOptions {
|
||||
|
||||
private val log = LogManager.getLogger(classOf[DefaultSource])
|
||||
|
||||
/**
|
||||
* The client operation, that this write should do
|
||||
* The write operation, that this write should do
|
||||
*
|
||||
* Default: upsert()
|
||||
*/
|
||||
@@ -91,16 +125,35 @@ object DataSourceWriteOptions {
|
||||
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL
|
||||
|
||||
/**
|
||||
* The storage type for the underlying data, for this write.
|
||||
* The table type for the underlying data, for this write.
|
||||
* Note that this can't change across writes.
|
||||
*
|
||||
* Default: COPY_ON_WRITE
|
||||
*/
|
||||
val TABLE_TYPE_OPT_KEY = "hoodie.datasource.write.table.type"
|
||||
val COW_TABLE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
|
||||
val MOR_TABLE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
|
||||
val DEFAULT_TABLE_TYPE_OPT_VAL = COW_TABLE_TYPE_OPT_VAL
|
||||
|
||||
@Deprecated
|
||||
val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
|
||||
@Deprecated
|
||||
val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
|
||||
@Deprecated
|
||||
val MOR_STORAGE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
|
||||
@Deprecated
|
||||
val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
|
||||
|
||||
def translateStorageTypeToTableType(optParams: Map[String, String]) : Map[String, String] = {
|
||||
if (optParams.contains(STORAGE_TYPE_OPT_KEY) && !optParams.contains(TABLE_TYPE_OPT_KEY)) {
|
||||
log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a later release; Please use " + TABLE_TYPE_OPT_KEY)
|
||||
optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY))
|
||||
} else {
|
||||
optParams
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Hive table name, to register the table into.
|
||||
*
|
||||
|
||||
@@ -50,20 +50,14 @@ class DefaultSource extends RelationProvider
|
||||
optParams: Map[String, String],
|
||||
schema: StructType): BaseRelation = {
|
||||
// Add default options for unspecified read options keys.
|
||||
val parameters = Map(VIEW_TYPE_OPT_KEY -> DEFAULT_VIEW_TYPE_OPT_VAL) ++ optParams
|
||||
val parameters = Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL) ++ translateViewTypesToQueryTypes(optParams)
|
||||
|
||||
val path = parameters.get("path")
|
||||
if (path.isEmpty) {
|
||||
throw new HoodieException("'path' must be specified.")
|
||||
}
|
||||
|
||||
if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_REALTIME_OPT_VAL)) {
|
||||
throw new HoodieException("Realtime view not supported yet via data source. Please use HiveContext route.")
|
||||
}
|
||||
|
||||
if (parameters(VIEW_TYPE_OPT_KEY).equals(VIEW_TYPE_INCREMENTAL_OPT_VAL)) {
|
||||
new IncrementalRelation(sqlContext, path.get, optParams, schema)
|
||||
} else {
|
||||
if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_SNAPSHOT_OPT_VAL)) {
|
||||
// this is just effectively RO view only, where `path` can contain a mix of
|
||||
// non-hoodie/hoodie path files. set the path filter up
|
||||
sqlContext.sparkContext.hadoopConfiguration.setClass(
|
||||
@@ -72,6 +66,8 @@ class DefaultSource extends RelationProvider
|
||||
classOf[org.apache.hadoop.fs.PathFilter]);
|
||||
|
||||
log.info("Constructing hoodie (as parquet) data source with options :" + parameters)
|
||||
log.warn("Snapshot view not supported yet via data source, for MERGE_ON_READ tables. " +
|
||||
"Please query the Hive table registered using Spark SQL.")
|
||||
// simply return as a regular parquet relation
|
||||
DataSource.apply(
|
||||
sparkSession = sqlContext.sparkSession,
|
||||
@@ -79,6 +75,10 @@ class DefaultSource extends RelationProvider
|
||||
className = "parquet",
|
||||
options = parameters)
|
||||
.resolveRelation()
|
||||
} else if (parameters(QUERY_TYPE_OPT_KEY).equals(QUERY_TYPE_INCREMENTAL_OPT_VAL)) {
|
||||
new IncrementalRelation(sqlContext, path.get, optParams, schema)
|
||||
} else {
|
||||
throw new HoodieException("Invalid query type :" + parameters(QUERY_TYPE_OPT_KEY))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -58,7 +58,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") =>
|
||||
case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer")
|
||||
}
|
||||
val storageType = parameters(STORAGE_TYPE_OPT_KEY)
|
||||
val tableType = parameters(TABLE_TYPE_OPT_KEY)
|
||||
val operation =
|
||||
// It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true
|
||||
// Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly
|
||||
@@ -119,7 +119,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
|
||||
// Create the table if not present
|
||||
if (!exists) {
|
||||
HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, storageType,
|
||||
HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, tableType,
|
||||
tblName.get, "archived", parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
}
|
||||
|
||||
@@ -192,7 +192,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_OPT_VAL,
|
||||
STORAGE_TYPE_OPT_KEY -> DEFAULT_STORAGE_TYPE_OPT_VAL,
|
||||
TABLE_TYPE_OPT_KEY -> DEFAULT_TABLE_TYPE_OPT_VAL,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> DEFAULT_PRECOMBINE_FIELD_OPT_VAL,
|
||||
PAYLOAD_CLASS_OPT_KEY -> DEFAULT_PAYLOAD_OPT_VAL,
|
||||
RECORDKEY_FIELD_OPT_KEY -> DEFAULT_RECORDKEY_FIELD_OPT_VAL,
|
||||
@@ -212,7 +212,7 @@ private[hudi] object HoodieSparkSqlWriter {
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL,
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL
|
||||
) ++ parameters
|
||||
) ++ translateStorageTypeToTableType(parameters)
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
|
||||
@@ -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