[HUDI-89] Add configOption & refactor all configs based on that (#2833)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -71,7 +71,7 @@ public class HoodieDatasetBulkInsertHelper {
|
||||
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key());
|
||||
BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
|
||||
StructType structTypeForUDF = rows.schema();
|
||||
|
||||
|
||||
@@ -71,10 +71,10 @@ class DefaultSource extends RelationProvider
|
||||
optParams: Map[String, String],
|
||||
schema: StructType): BaseRelation = {
|
||||
// Add default options for unspecified read options keys.
|
||||
val parameters = translateViewTypesToQueryTypes(optParams)
|
||||
val parameters = DataSourceOptionsHelper.translateConfigurations(optParams)
|
||||
|
||||
val path = parameters.get("path")
|
||||
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY)
|
||||
val readPathsStr = parameters.get(DataSourceReadOptions.READ_PATHS_OPT_KEY.key)
|
||||
if (path.isEmpty && readPathsStr.isEmpty) {
|
||||
throw new HoodieException(s"'path' or '$READ_PATHS_OPT_KEY' or both must be specified.")
|
||||
}
|
||||
@@ -85,10 +85,10 @@ class DefaultSource extends RelationProvider
|
||||
val fs = FSUtils.getFs(allPaths.head, sqlContext.sparkContext.hadoopConfiguration)
|
||||
// Use the HoodieFileIndex only if the 'path' is not globbed.
|
||||
// Or else we use the original way to read hoodie table.
|
||||
val enableFileIndex = optParams.get(ENABLE_HOODIE_FILE_INDEX)
|
||||
.map(_.toBoolean).getOrElse(DEFAULT_ENABLE_HOODIE_FILE_INDEX)
|
||||
val enableFileIndex = optParams.get(ENABLE_HOODIE_FILE_INDEX.key)
|
||||
.map(_.toBoolean).getOrElse(ENABLE_HOODIE_FILE_INDEX.defaultValue)
|
||||
val useHoodieFileIndex = enableFileIndex && path.isDefined && !path.get.contains("*") &&
|
||||
!parameters.contains(DataSourceReadOptions.READ_PATHS_OPT_KEY)
|
||||
!parameters.contains(DataSourceReadOptions.READ_PATHS_OPT_KEY.key)
|
||||
val globPaths = if (useHoodieFileIndex) {
|
||||
None
|
||||
} else {
|
||||
@@ -105,7 +105,7 @@ class DefaultSource extends RelationProvider
|
||||
val metaClient = HoodieTableMetaClient.builder().setConf(fs.getConf).setBasePath(tablePath).build()
|
||||
val isBootstrappedTable = metaClient.getTableConfig.getBootstrapBasePath.isPresent
|
||||
val tableType = metaClient.getTableType
|
||||
val queryType = parameters(QUERY_TYPE_OPT_KEY)
|
||||
val queryType = parameters.getOrElse(QUERY_TYPE_OPT_KEY.key, QUERY_TYPE_OPT_KEY.defaultValue)
|
||||
log.info(s"Is bootstrapped table => $isBootstrappedTable, tableType is: $tableType")
|
||||
|
||||
(tableType, queryType, isBootstrappedTable) match {
|
||||
@@ -157,7 +157,7 @@ class DefaultSource extends RelationProvider
|
||||
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
|
||||
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
|
||||
|
||||
if (translatedOptions(OPERATION_OPT_KEY).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
|
||||
if (translatedOptions(OPERATION_OPT_KEY.key).equals(BOOTSTRAP_OPERATION_OPT_VAL)) {
|
||||
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, translatedOptions, dfWithoutMetaCols)
|
||||
} else {
|
||||
HoodieSparkSqlWriter.write(sqlContext, mode, translatedOptions, dfWithoutMetaCols)
|
||||
|
||||
@@ -28,17 +28,18 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.avro.HoodieAvroUtils
|
||||
import org.apache.hudi.client.HoodieWriteResult
|
||||
import org.apache.hudi.client.SparkRDDWriteClient
|
||||
import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
|
||||
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hive.util.ConfigUtils
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
@@ -53,8 +54,6 @@ import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode, SparkSession}
|
||||
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
import org.apache.hudi.common.table.HoodieTableConfig.{DEFAULT_ARCHIVELOG_FOLDER}
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
|
||||
object HoodieSparkSqlWriter {
|
||||
|
||||
@@ -75,22 +74,23 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.get("path")
|
||||
val tblNameOp = parameters.get(HoodieWriteConfig.TABLE_NAME)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
|
||||
asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
|
||||
if (path.isEmpty || tblNameOp.isEmpty) {
|
||||
throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}', 'path' must be set.")
|
||||
if (path.isEmpty) {
|
||||
throw new HoodieException(s"'path' must be set.")
|
||||
}
|
||||
val tblName = tblNameOp.get.trim
|
||||
val tblName = tblNameOp.trim
|
||||
sparkContext.getConf.getOption("spark.serializer") match {
|
||||
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 tableType = HoodieTableType.valueOf(parameters(TABLE_TYPE_OPT_KEY))
|
||||
var operation = WriteOperationType.fromValue(parameters(OPERATION_OPT_KEY))
|
||||
val tableType = HoodieTableType.valueOf(hoodieConfig.getString(TABLE_TYPE_OPT_KEY))
|
||||
var operation = WriteOperationType.fromValue(hoodieConfig.getString(OPERATION_OPT_KEY))
|
||||
// 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
|
||||
// or not set (in which case it will be set as "upsert" by parametersWithWriteDefaults()) .
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean &&
|
||||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS_OPT_KEY) &&
|
||||
operation == WriteOperationType.UPSERT) {
|
||||
|
||||
log.warn(s"$UPSERT_OPERATION_OPT_VAL is not applicable " +
|
||||
@@ -116,17 +116,15 @@ object HoodieSparkSqlWriter {
|
||||
handleSaveModes(mode, basePath, tableConfig, tblName, operation, fs)
|
||||
// Create the table if not present
|
||||
if (!tableExists) {
|
||||
val archiveLogFolder = parameters.getOrElse(
|
||||
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER)
|
||||
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
|
||||
val partitionColumns = HoodieWriterUtils.getPartitionColumns(keyGenerator)
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(tableType)
|
||||
.setTableName(tblName)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
|
||||
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
|
||||
.setPartitionColumns(partitionColumns)
|
||||
.initTable(sparkContext.hadoopConfiguration, path.get)
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
@@ -136,7 +134,7 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
// short-circuit if bulk_insert via row is enabled.
|
||||
// scalastyle:off
|
||||
if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean &&
|
||||
if (hoodieConfig.getBoolean(ENABLE_ROW_WRITER_OPT_KEY) &&
|
||||
operation == WriteOperationType.BULK_INSERT) {
|
||||
val (success, commitTime: common.util.Option[String]) = bulkInsertAsRow(sqlContext, parameters, df, tblName,
|
||||
basePath, path, instantTime)
|
||||
@@ -157,23 +155,23 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
// Convert to RDD[HoodieRecord]
|
||||
val genericRecords: RDD[GenericRecord] = HoodieSparkUtils.createRdd(df, schema, structName, nameSpace)
|
||||
val shouldCombine = parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean || operation.equals(WriteOperationType.UPSERT)
|
||||
val shouldCombine = parameters(INSERT_DROP_DUPS_OPT_KEY.key()).toBoolean || operation.equals(WriteOperationType.UPSERT);
|
||||
val hoodieAllIncomingRecords = genericRecords.map(gr => {
|
||||
val hoodieRecord = if (shouldCombine) {
|
||||
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, parameters(PRECOMBINE_FIELD_OPT_KEY), false)
|
||||
val orderingVal = HoodieAvroUtils.getNestedFieldVal(gr, hoodieConfig.getString(PRECOMBINE_FIELD_OPT_KEY), false)
|
||||
.asInstanceOf[Comparable[_]]
|
||||
DataSourceUtils.createHoodieRecord(gr,
|
||||
orderingVal, keyGenerator.getKey(gr),
|
||||
parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
|
||||
} else {
|
||||
DataSourceUtils.createHoodieRecord(gr, keyGenerator.getKey(gr), parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
DataSourceUtils.createHoodieRecord(gr, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_OPT_KEY))
|
||||
}
|
||||
hoodieRecord
|
||||
}).toJavaRDD()
|
||||
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key)
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -181,7 +179,7 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
val hoodieRecords =
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
|
||||
if (hoodieConfig.getBoolean(INSERT_DROP_DUPS_OPT_KEY)) {
|
||||
DataSourceUtils.dropDuplicates(jsc, hoodieAllIncomingRecords, mapAsJavaMap(parameters))
|
||||
} else {
|
||||
hoodieAllIncomingRecords
|
||||
@@ -212,7 +210,7 @@ object HoodieSparkSqlWriter {
|
||||
// Create a HoodieWriteClient & issue the delete.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
|
||||
null, path.get, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)))
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -260,13 +258,13 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.getOrElse("path", throw new HoodieException("'path' must be set."))
|
||||
val tableName = parameters.getOrElse(HoodieWriteConfig.TABLE_NAME,
|
||||
throw new HoodieException(s"'${HoodieWriteConfig.TABLE_NAME}' must be set."))
|
||||
val tableType = parameters(TABLE_TYPE_OPT_KEY)
|
||||
val bootstrapBasePath = parameters.getOrElse(BOOTSTRAP_BASE_PATH_PROP,
|
||||
throw new HoodieException(s"'${BOOTSTRAP_BASE_PATH_PROP}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
|
||||
" operation'"))
|
||||
val bootstrapIndexClass = parameters.getOrDefault(BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
|
||||
val tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE_OPT_KEY)
|
||||
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BOOTSTRAP_BASE_PATH_PROP,
|
||||
s"'${BOOTSTRAP_BASE_PATH_PROP.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
|
||||
" operation'")
|
||||
val bootstrapIndexClass = hoodieConfig.getStringOrDefault(BOOTSTRAP_INDEX_CLASS_PROP)
|
||||
|
||||
var schema: String = null
|
||||
if (df.schema.nonEmpty) {
|
||||
@@ -290,15 +288,14 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
if (!tableExists) {
|
||||
val archiveLogFolder = parameters.getOrElse(
|
||||
HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER)
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
|
||||
val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.valueOf(tableType))
|
||||
.setTableName(tableName)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(parameters(PAYLOAD_CLASS_OPT_KEY))
|
||||
.setPreCombineField(parameters.getOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
|
||||
.setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS_OPT_KEY))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD_OPT_KEY, null))
|
||||
.setBootstrapIndexClass(bootstrapIndexClass)
|
||||
.setBootstrapBasePath(bootstrapBasePath)
|
||||
.setPartitionColumns(partitionColumns)
|
||||
@@ -313,7 +310,7 @@ object HoodieSparkSqlWriter {
|
||||
} finally {
|
||||
writeClient.close()
|
||||
}
|
||||
val metaSyncSuccess = metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
|
||||
val metaSyncSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
metaSyncSuccess
|
||||
}
|
||||
|
||||
@@ -333,10 +330,10 @@ object HoodieSparkSqlWriter {
|
||||
val schema = AvroConversionUtils.convertStructTypeToAvroSchema(df.schema, structName, nameSpace)
|
||||
sparkContext.getConf.registerAvroSchemas(schema)
|
||||
log.info(s"Registered avro schema : ${schema.toString(true)}")
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY).toBoolean) {
|
||||
if (parameters(INSERT_DROP_DUPS_OPT_KEY.key).toBoolean) {
|
||||
throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
|
||||
}
|
||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA, schema.toString)
|
||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA.key, schema.toString)
|
||||
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
|
||||
val hoodieDF = HoodieDatasetBulkInsertHelper.prepareHoodieDatasetForBulkInsert(sqlContext, writeConfig, df, structName, nameSpace)
|
||||
if (SPARK_VERSION.startsWith("2.")) {
|
||||
@@ -347,7 +344,7 @@ object HoodieSparkSqlWriter {
|
||||
} else if (SPARK_VERSION.startsWith("3.")) {
|
||||
hoodieDF.write.format("org.apache.hudi.spark3.internal")
|
||||
.option(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY, instantTime)
|
||||
.option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL, hoodieDF.schema.toDDL)
|
||||
.option(HoodieWriteConfig.BULKINSERT_INPUT_DATA_SCHEMA_DDL.key, hoodieDF.schema.toDDL)
|
||||
.options(params)
|
||||
.mode(SaveMode.Append)
|
||||
.save()
|
||||
@@ -355,11 +352,12 @@ object HoodieSparkSqlWriter {
|
||||
throw new HoodieException("Bulk insert using row writer is not supported with current Spark version."
|
||||
+ " To use row writer please switch to spark 2 or spark 3")
|
||||
}
|
||||
val hiveSyncEnabled = params.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val metaSyncEnabled = params.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params)
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED_OPT_KEY).toBoolean
|
||||
val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED_OPT_KEY).toBoolean
|
||||
val syncHiveSuccess =
|
||||
if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(sqlContext.sparkSession, parameters, basePath, df.schema)
|
||||
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
@@ -399,47 +397,44 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
}
|
||||
|
||||
private def syncHive(basePath: Path, fs: FileSystem, parameters: Map[String, String]): Boolean = {
|
||||
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, parameters)
|
||||
private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig): Boolean = {
|
||||
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig)
|
||||
val hiveConf: HiveConf = new HiveConf()
|
||||
hiveConf.addResource(fs.getConf)
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
|
||||
true
|
||||
}
|
||||
|
||||
private def buildSyncConfig(basePath: Path, parameters: Map[String, String]): HiveSyncConfig = {
|
||||
private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig): HiveSyncConfig = {
|
||||
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
|
||||
hiveSyncConfig.basePath = basePath.toString
|
||||
hiveSyncConfig.baseFileFormat = parameters(HIVE_BASE_FILE_FORMAT_OPT_KEY)
|
||||
hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT_OPT_KEY)
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
parameters.get(HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.databaseName = parameters(HIVE_DATABASE_OPT_KEY)
|
||||
hiveSyncConfig.tableName = parameters(HIVE_TABLE_OPT_KEY)
|
||||
hiveSyncConfig.hiveUser = parameters(HIVE_USER_OPT_KEY)
|
||||
hiveSyncConfig.hivePass = parameters(HIVE_PASS_OPT_KEY)
|
||||
hiveSyncConfig.jdbcUrl = parameters(HIVE_URL_OPT_KEY)
|
||||
hiveSyncConfig.skipROSuffix = parameters.getOrElse(HIVE_SKIP_RO_SUFFIX,
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL).toBoolean
|
||||
hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY).toBoolean
|
||||
hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE_OPT_KEY)
|
||||
hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE_OPT_KEY)
|
||||
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER_OPT_KEY)
|
||||
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS_OPT_KEY)
|
||||
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL_OPT_KEY)
|
||||
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX,
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.defaultValue).toBoolean
|
||||
hiveSyncConfig.partitionFields =
|
||||
ListBuffer(parameters(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = parameters(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY)
|
||||
hiveSyncConfig.useJdbc = parameters(HIVE_USE_JDBC_OPT_KEY).toBoolean
|
||||
hiveSyncConfig.useFileListingFromMetadata = parameters(HoodieMetadataConfig.METADATA_ENABLE_PROP).toBoolean
|
||||
hiveSyncConfig.verifyMetadataFileListing = parameters(HoodieMetadataConfig.METADATA_VALIDATE_PROP).toBoolean
|
||||
hiveSyncConfig.ignoreExceptions = parameters.get(HIVE_IGNORE_EXCEPTIONS_OPT_KEY).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.supportTimestamp = parameters.get(HIVE_SUPPORT_TIMESTAMP).exists(r => r.toBoolean)
|
||||
hiveSyncConfig.autoCreateDatabase = parameters.getOrElse(HIVE_AUTO_CREATE_DATABASE_OPT_KEY,
|
||||
DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY).toBoolean
|
||||
hiveSyncConfig.decodePartition = parameters.getOrElse(URL_ENCODE_PARTITIONING_OPT_KEY,
|
||||
DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL).toBoolean
|
||||
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS_OPT_KEY).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY)
|
||||
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC_OPT_KEY)
|
||||
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_ENABLE_PROP)
|
||||
hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_VALIDATE_PROP)
|
||||
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS_OPT_KEY).toBoolean
|
||||
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP).toBoolean
|
||||
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE_OPT_KEY).toBoolean
|
||||
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING_OPT_KEY).toBoolean
|
||||
|
||||
val syncAsDtaSourceTable = parameters.getOrElse(DataSourceWriteOptions.HIVE_SYNC_AS_DATA_SOURCE_TABLE,
|
||||
DataSourceWriteOptions.DEFAULT_HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
val syncAsDtaSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
if (syncAsDtaSourceTable) {
|
||||
hiveSyncConfig.tableProperties = parameters.getOrElse(HIVE_TABLE_PROPERTIES, null)
|
||||
val serdePropText = createSqlTableSerdeProperties(parameters, basePath.toString)
|
||||
hiveSyncConfig.tableProperties = hoodieConfig.getStringOrDefault(HIVE_TABLE_PROPERTIES, null)
|
||||
val serdePropText = createSqlTableSerdeProperties(hoodieConfig, basePath.toString)
|
||||
val serdeProp = ConfigUtils.toMap(serdePropText)
|
||||
serdeProp.put(ConfigUtils.SPARK_QUERY_TYPE_KEY, DataSourceReadOptions.QUERY_TYPE_OPT_KEY)
|
||||
serdeProp.put(ConfigUtils.SPARK_QUERY_TYPE_KEY, DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key)
|
||||
serdeProp.put(ConfigUtils.SPARK_QUERY_AS_RO_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
serdeProp.put(ConfigUtils.SPARK_QUERY_AS_RT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
|
||||
@@ -453,18 +448,18 @@ object HoodieSparkSqlWriter {
|
||||
* Add Spark Sql related table properties to the HIVE_TABLE_PROPERTIES.
|
||||
* @param sqlConf The spark sql conf.
|
||||
* @param schema The schema to write to the table.
|
||||
* @param parameters The origin parameters.
|
||||
* @param hoodieConfig The HoodieConfig contains origin parameters.
|
||||
* @return A new parameters added the HIVE_TABLE_PROPERTIES property.
|
||||
*/
|
||||
private def addSqlTableProperties(sqlConf: SQLConf, schema: StructType,
|
||||
parameters: Map[String, String]): Map[String, String] = {
|
||||
hoodieConfig: HoodieConfig): HoodieConfig = {
|
||||
// Convert the schema and partition info used by spark sql to hive table properties.
|
||||
// The following code refers to the spark code in
|
||||
// https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
|
||||
|
||||
// Sync schema with meta fields
|
||||
val schemaWithMetaFields = HoodieSqlUtils.addMetaFields(schema)
|
||||
val partitionSet = parameters(HIVE_PARTITION_FIELDS_OPT_KEY)
|
||||
val partitionSet = hoodieConfig.getString(HIVE_PARTITION_FIELDS_OPT_KEY)
|
||||
.split(",").map(_.trim).filter(!_.isEmpty).toSet
|
||||
val threshold = sqlConf.getConf(SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
|
||||
@@ -487,31 +482,32 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
}
|
||||
var sqlPropertyText = ConfigUtils.configToString(properties)
|
||||
sqlPropertyText = if (parameters.containsKey(HIVE_TABLE_PROPERTIES)) {
|
||||
sqlPropertyText + "\n" + parameters(HIVE_TABLE_PROPERTIES)
|
||||
sqlPropertyText = if (hoodieConfig.contains(HIVE_TABLE_PROPERTIES)) {
|
||||
sqlPropertyText + "\n" + hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
|
||||
} else {
|
||||
sqlPropertyText
|
||||
}
|
||||
parameters + (HIVE_TABLE_PROPERTIES -> sqlPropertyText)
|
||||
hoodieConfig.setValue(HIVE_TABLE_PROPERTIES, sqlPropertyText)
|
||||
hoodieConfig
|
||||
}
|
||||
|
||||
private def createSqlTableSerdeProperties(parameters: Map[String, String], basePath: String): String = {
|
||||
private def createSqlTableSerdeProperties(hoodieConfig: HoodieConfig, basePath: String): String = {
|
||||
val pathProp = s"path=$basePath"
|
||||
if (parameters.containsKey(HIVE_TABLE_SERDE_PROPERTIES)) {
|
||||
pathProp + "\n" + parameters(HIVE_TABLE_SERDE_PROPERTIES)
|
||||
if (hoodieConfig.contains(HIVE_TABLE_SERDE_PROPERTIES)) {
|
||||
pathProp + "\n" + hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
|
||||
} else {
|
||||
pathProp
|
||||
}
|
||||
}
|
||||
|
||||
private def metaSync(spark: SparkSession, parameters: Map[String, String], basePath: Path,
|
||||
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
|
||||
schema: StructType): Boolean = {
|
||||
val hiveSyncEnabled = parameters.get(HIVE_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
var metaSyncEnabled = parameters.get(META_SYNC_ENABLED_OPT_KEY).exists(r => r.toBoolean)
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED_OPT_KEY).toBoolean
|
||||
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED_OPT_KEY).toBoolean
|
||||
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||
parameters(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
|
||||
val newParameters = addSqlTableProperties(spark.sessionState.conf, schema, parameters)
|
||||
val newHoodieConfig = addSqlTableProperties(spark.sessionState.conf, schema, hoodieConfig)
|
||||
// for backward compatibility
|
||||
if (hiveSyncEnabled) {
|
||||
metaSyncEnabled = true
|
||||
@@ -523,13 +519,13 @@ object HoodieSparkSqlWriter {
|
||||
syncClientToolClassSet.foreach(impl => {
|
||||
val syncSuccess = impl.trim match {
|
||||
case "org.apache.hudi.hive.HiveSyncTool" => {
|
||||
log.info("Syncing to Hive Metastore (URL: " + parameters(HIVE_URL_OPT_KEY) + ")")
|
||||
syncHive(basePath, fs, newParameters)
|
||||
log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL_OPT_KEY) + ")")
|
||||
syncHive(basePath, fs, newHoodieConfig)
|
||||
true
|
||||
}
|
||||
case _ => {
|
||||
val properties = new Properties();
|
||||
properties.putAll(newParameters)
|
||||
val properties = new Properties()
|
||||
properties.putAll(newHoodieConfig.getProps)
|
||||
properties.put("basePath", basePath.toString)
|
||||
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
|
||||
syncHoodie.syncHoodieTable()
|
||||
@@ -559,7 +555,7 @@ object HoodieSparkSqlWriter {
|
||||
if(writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors).isEmpty()) {
|
||||
log.info("Proceeding to commit the write.")
|
||||
val metaMap = parameters.filter(kv =>
|
||||
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY)))
|
||||
kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY.key)))
|
||||
val commitSuccess =
|
||||
client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses,
|
||||
common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))),
|
||||
@@ -583,7 +579,8 @@ object HoodieSparkSqlWriter {
|
||||
|
||||
log.info(s"Compaction Scheduled is $compactionInstant")
|
||||
|
||||
val metaSyncSuccess = metaSync(spark, parameters, tableInstantInfo.basePath, schema)
|
||||
val metaSyncSuccess = metaSync(spark, HoodieWriterUtils.convertMapToHoodieConfig(parameters),
|
||||
tableInstantInfo.basePath, schema)
|
||||
|
||||
log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled")
|
||||
if (!asyncCompactionEnabled) {
|
||||
@@ -613,7 +610,7 @@ object HoodieSparkSqlWriter {
|
||||
parameters: Map[String, String], configuration: Configuration) : Boolean = {
|
||||
log.info(s"Config.inlineCompactionEnabled ? ${client.getConfig.inlineCompactionEnabled}")
|
||||
if (asyncCompactionTriggerFnDefined && !client.getConfig.inlineCompactionEnabled
|
||||
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY).exists(r => r.toBoolean)) {
|
||||
&& parameters.get(ASYNC_COMPACT_ENABLE_OPT_KEY.key).exists(r => r.toBoolean)) {
|
||||
tableConfig.getTableType == HoodieTableType.MERGE_ON_READ
|
||||
} else {
|
||||
false
|
||||
|
||||
@@ -47,9 +47,9 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
|
||||
private val log = LogManager.getLogger(classOf[HoodieStreamingSink])
|
||||
|
||||
private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT_OPT_KEY).toInt
|
||||
private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY).toLong
|
||||
private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY).toBoolean
|
||||
private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT_OPT_KEY.key).toInt
|
||||
private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS_OPT_KEY.key).toLong
|
||||
private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key).toBoolean
|
||||
|
||||
private var isAsyncCompactorServiceShutdownAbnormally = false
|
||||
|
||||
@@ -101,7 +101,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
log.error(s"Micro batch id=$batchId threw following exception: ", e)
|
||||
if (ignoreFailedBatch) {
|
||||
log.info(s"Ignore the exception and move on streaming as per " +
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration")
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key} configuration")
|
||||
Success((true, None, None))
|
||||
} else {
|
||||
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")
|
||||
@@ -115,7 +115,7 @@ class HoodieStreamingSink(sqlContext: SQLContext,
|
||||
}))
|
||||
if (ignoreFailedBatch) {
|
||||
log.info(s"Ignore the errors and move on streaming as per " +
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY} configuration")
|
||||
s"${DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key} configuration")
|
||||
Success((true, None, None))
|
||||
} else {
|
||||
if (retryCnt > 1) log.info(s"Retrying the failed micro batch id=$batchId ...")
|
||||
|
||||
@@ -17,14 +17,14 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.TypedProperties
|
||||
import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
|
||||
|
||||
import scala.collection.JavaConversions.mapAsJavaMap
|
||||
import scala.collection.JavaConverters.mapAsScalaMapConverter
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_ENABLE_PROP
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.METADATA_VALIDATE_PROP
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
@@ -46,36 +46,36 @@ object HoodieWriterUtils {
|
||||
* @return
|
||||
*/
|
||||
def parametersWithWriteDefaults(parameters: Map[String, String]): Map[String, String] = {
|
||||
Map(OPERATION_OPT_KEY -> DEFAULT_OPERATION_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,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> DEFAULT_PARTITIONPATH_FIELD_OPT_VAL,
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
METADATA_ENABLE_PROP -> DEFAULT_METADATA_ENABLE.toString,
|
||||
METADATA_VALIDATE_PROP -> DEFAULT_METADATA_VALIDATE.toString,
|
||||
COMMIT_METADATA_KEYPREFIX_OPT_KEY -> DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL,
|
||||
INSERT_DROP_DUPS_OPT_KEY -> DEFAULT_INSERT_DROP_DUPS_OPT_VAL,
|
||||
STREAMING_RETRY_CNT_OPT_KEY -> DEFAULT_STREAMING_RETRY_CNT_OPT_VAL,
|
||||
STREAMING_RETRY_INTERVAL_MS_OPT_KEY -> DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL,
|
||||
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY -> DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL,
|
||||
META_SYNC_CLIENT_TOOL_CLASS -> DEFAULT_META_SYNC_CLIENT_TOOL_CLASS,
|
||||
HIVE_SYNC_ENABLED_OPT_KEY -> DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL,
|
||||
META_SYNC_ENABLED_OPT_KEY -> DEFAULT_META_SYNC_ENABLED_OPT_VAL,
|
||||
HIVE_DATABASE_OPT_KEY -> DEFAULT_HIVE_DATABASE_OPT_VAL,
|
||||
HIVE_TABLE_OPT_KEY -> DEFAULT_HIVE_TABLE_OPT_VAL,
|
||||
HIVE_BASE_FILE_FORMAT_OPT_KEY -> DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL,
|
||||
HIVE_USER_OPT_KEY -> DEFAULT_HIVE_USER_OPT_VAL,
|
||||
HIVE_PASS_OPT_KEY -> DEFAULT_HIVE_PASS_OPT_VAL,
|
||||
HIVE_URL_OPT_KEY -> DEFAULT_HIVE_URL_OPT_VAL,
|
||||
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,
|
||||
HIVE_USE_JDBC_OPT_KEY -> DEFAULT_HIVE_USE_JDBC_OPT_VAL,
|
||||
ASYNC_COMPACT_ENABLE_OPT_KEY -> DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL,
|
||||
ENABLE_ROW_WRITER_OPT_KEY -> DEFAULT_ENABLE_ROW_WRITER_OPT_VAL
|
||||
) ++ translateStorageTypeToTableType(parameters)
|
||||
Map(OPERATION_OPT_KEY.key -> OPERATION_OPT_KEY.defaultValue,
|
||||
TABLE_TYPE_OPT_KEY.key -> TABLE_TYPE_OPT_KEY.defaultValue,
|
||||
PRECOMBINE_FIELD_OPT_KEY.key -> PRECOMBINE_FIELD_OPT_KEY.defaultValue,
|
||||
PAYLOAD_CLASS_OPT_KEY.key -> PAYLOAD_CLASS_OPT_KEY.defaultValue,
|
||||
RECORDKEY_FIELD_OPT_KEY.key -> RECORDKEY_FIELD_OPT_KEY.defaultValue,
|
||||
PARTITIONPATH_FIELD_OPT_KEY.key -> PARTITIONPATH_FIELD_OPT_KEY.defaultValue,
|
||||
KEYGENERATOR_CLASS_OPT_KEY.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
METADATA_ENABLE_PROP.key -> METADATA_ENABLE_PROP.defaultValue.toString,
|
||||
METADATA_VALIDATE_PROP.key -> METADATA_VALIDATE_PROP.defaultValue.toString,
|
||||
COMMIT_METADATA_KEYPREFIX_OPT_KEY.key -> COMMIT_METADATA_KEYPREFIX_OPT_KEY.defaultValue,
|
||||
INSERT_DROP_DUPS_OPT_KEY.key -> INSERT_DROP_DUPS_OPT_KEY.defaultValue,
|
||||
STREAMING_RETRY_CNT_OPT_KEY.key -> STREAMING_RETRY_CNT_OPT_KEY.defaultValue,
|
||||
STREAMING_RETRY_INTERVAL_MS_OPT_KEY.key -> STREAMING_RETRY_INTERVAL_MS_OPT_KEY.defaultValue,
|
||||
STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.key -> STREAMING_IGNORE_FAILED_BATCH_OPT_KEY.defaultValue,
|
||||
META_SYNC_CLIENT_TOOL_CLASS.key -> META_SYNC_CLIENT_TOOL_CLASS.defaultValue,
|
||||
HIVE_SYNC_ENABLED_OPT_KEY.key -> HIVE_SYNC_ENABLED_OPT_KEY.defaultValue,
|
||||
META_SYNC_ENABLED_OPT_KEY.key -> META_SYNC_ENABLED_OPT_KEY.defaultValue,
|
||||
HIVE_DATABASE_OPT_KEY.key -> HIVE_DATABASE_OPT_KEY.defaultValue,
|
||||
HIVE_TABLE_OPT_KEY.key -> HIVE_TABLE_OPT_KEY.defaultValue,
|
||||
HIVE_BASE_FILE_FORMAT_OPT_KEY.key -> HIVE_BASE_FILE_FORMAT_OPT_KEY.defaultValue,
|
||||
HIVE_USER_OPT_KEY.key -> HIVE_USER_OPT_KEY.defaultValue,
|
||||
HIVE_PASS_OPT_KEY.key -> HIVE_PASS_OPT_KEY.defaultValue,
|
||||
HIVE_URL_OPT_KEY.key -> HIVE_URL_OPT_KEY.defaultValue,
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY.key -> HIVE_PARTITION_FIELDS_OPT_KEY.defaultValue,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.defaultValue,
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue,
|
||||
HIVE_USE_JDBC_OPT_KEY.key -> HIVE_USE_JDBC_OPT_KEY.defaultValue,
|
||||
ASYNC_COMPACT_ENABLE_OPT_KEY.key -> ASYNC_COMPACT_ENABLE_OPT_KEY.defaultValue,
|
||||
ENABLE_ROW_WRITER_OPT_KEY.key -> ENABLE_ROW_WRITER_OPT_KEY.defaultValue
|
||||
) ++ DataSourceOptionsHelper.translateConfigurations(parameters)
|
||||
}
|
||||
|
||||
def toProperties(params: Map[String, String]): TypedProperties = {
|
||||
@@ -111,4 +111,10 @@ object HoodieWriterUtils {
|
||||
case _=> null
|
||||
}
|
||||
}
|
||||
|
||||
def convertMapToHoodieConfig(parameters: Map[String, String]): HoodieConfig = {
|
||||
val properties = new Properties()
|
||||
properties.putAll(mapAsJavaMap(parameters))
|
||||
new HoodieConfig(properties)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -58,19 +58,19 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
}
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key)) {
|
||||
throw new HoodieException(s"Specify the begin instant time to pull from using " +
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key}")
|
||||
}
|
||||
|
||||
val useEndInstantSchema = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_VAL).toBoolean
|
||||
val useEndInstantSchema = optParams.getOrElse(DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY.key,
|
||||
DataSourceReadOptions.INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY.defaultValue).toBoolean
|
||||
|
||||
private val lastInstant = commitTimeline.lastInstant().get()
|
||||
|
||||
private val commitsToReturn = commitTimeline.findInstantsInRange(
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, lastInstant.getTimestamp))
|
||||
.getInstants.iterator().toList
|
||||
|
||||
// use schema from a file produced in the end/latest instant
|
||||
@@ -87,8 +87,8 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
StructType(skeletonSchema.fields ++ dataSchema.fields)
|
||||
}
|
||||
|
||||
private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL).split(",").filter(!_.isEmpty)
|
||||
private val filters = optParams.getOrElse(DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY.key,
|
||||
DataSourceReadOptions.PUSH_DOWN_INCR_FILTERS_OPT_KEY.defaultValue).split(",").filter(!_.isEmpty)
|
||||
|
||||
override def schema: StructType = usedSchema
|
||||
|
||||
@@ -114,10 +114,10 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
}
|
||||
|
||||
val pathGlobPattern = optParams.getOrElse(
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key,
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)
|
||||
val (filteredRegularFullPaths, filteredMetaBootstrapFullPaths) = {
|
||||
if(!pathGlobPattern.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) {
|
||||
if(!pathGlobPattern.equals(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)) {
|
||||
val globMatcher = new GlobPattern("*" + pathGlobPattern)
|
||||
(regularFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values,
|
||||
metaBootstrapFileIdToFullPath.filter(p => globMatcher.matches(p._2)).values)
|
||||
@@ -140,7 +140,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
|
||||
df = sqlContext.sparkSession.read
|
||||
.format("hudi")
|
||||
.schema(usedSchema)
|
||||
.option(DataSourceReadOptions.READ_PATHS_OPT_KEY, filteredMetaBootstrapFullPaths.mkString(","))
|
||||
.option(DataSourceReadOptions.READ_PATHS_OPT_KEY.key, filteredMetaBootstrapFullPaths.mkString(","))
|
||||
.load()
|
||||
}
|
||||
|
||||
|
||||
@@ -57,19 +57,19 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
|
||||
if (commitTimeline.empty()) {
|
||||
throw new HoodieException("No instants to incrementally pull")
|
||||
}
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY)) {
|
||||
if (!optParams.contains(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key)) {
|
||||
throw new HoodieException(s"Specify the begin instant time to pull from using " +
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY}")
|
||||
s"option ${DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key}")
|
||||
}
|
||||
|
||||
private val lastInstant = commitTimeline.lastInstant().get()
|
||||
private val mergeType = optParams.getOrElse(
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key,
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.defaultValue)
|
||||
|
||||
private val commitsTimelineToReturn = commitTimeline.findInstantsInRange(
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, lastInstant.getTimestamp))
|
||||
optParams(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key),
|
||||
optParams.getOrElse(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, lastInstant.getTimestamp))
|
||||
log.debug(s"${commitsTimelineToReturn.getInstants.iterator().toList.map(f => f.toString).mkString(",")}")
|
||||
private val commitsToReturn = commitsTimelineToReturn.getInstants.iterator().toList
|
||||
private val schemaUtil = new TableSchemaResolver(metaClient)
|
||||
@@ -84,7 +84,7 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
|
||||
} else {
|
||||
// get preCombineFiled from the options if this is a old table which have not store
|
||||
// the field to hoodie.properties
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD)
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
|
||||
}
|
||||
}
|
||||
override def schema: StructType = tableStructSchema
|
||||
@@ -178,10 +178,10 @@ class MergeOnReadIncrementalRelation(val sqlContext: SQLContext,
|
||||
|
||||
// Filter files based on user defined glob pattern
|
||||
val pathGlobPattern = optParams.getOrElse(
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key,
|
||||
DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)
|
||||
val filteredFileGroup = if(!pathGlobPattern
|
||||
.equals(DataSourceReadOptions.DEFAULT_INCR_PATH_GLOB_OPT_VAL)) {
|
||||
.equals(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.defaultValue)) {
|
||||
val globMatcher = new GlobPattern("*" + pathGlobPattern)
|
||||
fileGroup.filter(f => {
|
||||
if (f.getLatestFileSlice.get().getBaseFile.isPresent) {
|
||||
|
||||
@@ -74,8 +74,8 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
|
||||
private lazy val tableStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
|
||||
private val mergeType = optParams.getOrElse(
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY,
|
||||
DataSourceReadOptions.DEFAULT_REALTIME_MERGE_OPT_VAL)
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key,
|
||||
DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.defaultValue)
|
||||
private val maxCompactionMemoryInBytes = getMaxCompactionMemoryInBytes(jobConf)
|
||||
private val preCombineField = {
|
||||
val preCombineFieldFromTableConfig = metaClient.getTableConfig.getPreCombineField
|
||||
@@ -84,7 +84,7 @@ class MergeOnReadSnapshotRelation(val sqlContext: SQLContext,
|
||||
} else {
|
||||
// get preCombineFiled from the options if this is a old table which have not store
|
||||
// the field to hoodie.properties
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD)
|
||||
optParams.get(DataSourceReadOptions.READ_PRE_COMBINE_FIELD.key)
|
||||
}
|
||||
}
|
||||
override def schema: StructType = tableStructSchema
|
||||
|
||||
@@ -41,27 +41,27 @@ object HoodieOptionConfig {
|
||||
|
||||
val SQL_KEY_TABLE_PRIMARY_KEY: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("primaryKey")
|
||||
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)
|
||||
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
|
||||
.build()
|
||||
|
||||
val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("type")
|
||||
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME)
|
||||
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP.key)
|
||||
.defaultValue(SQL_VALUE_TABLE_TYPE_COW)
|
||||
.build()
|
||||
|
||||
val SQL_KEY_PRECOMBINE_FIELD: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("preCombineField")
|
||||
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)
|
||||
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
|
||||
.build()
|
||||
|
||||
val SQL_PAYLOAD_CLASS: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("payloadClass")
|
||||
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME)
|
||||
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key)
|
||||
.defaultValue(classOf[DefaultHoodieRecordPayload].getName)
|
||||
.build()
|
||||
|
||||
@@ -151,7 +151,7 @@ object HoodieOptionConfig {
|
||||
*/
|
||||
def getPrimaryColumns(options: Map[String, String]): Array[String] = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.get(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
params.get(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key)
|
||||
.map(_.split(",").filter(_.length > 0))
|
||||
.getOrElse(Array.empty)
|
||||
}
|
||||
@@ -163,13 +163,13 @@ object HoodieOptionConfig {
|
||||
*/
|
||||
def getTableType(options: Map[String, String]): String = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.getOrElse(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY,
|
||||
DataSourceWriteOptions.DEFAULT_TABLE_TYPE_OPT_VAL)
|
||||
params.getOrElse(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.defaultValue)
|
||||
}
|
||||
|
||||
def getPreCombineField(options: Map[String, String]): Option[String] = {
|
||||
val params = mappingSqlOptionToHoodieParam(options)
|
||||
params.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY)
|
||||
params.get(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key)
|
||||
}
|
||||
|
||||
def buildConf[T](): HoodieOptions[T] = {
|
||||
|
||||
@@ -44,7 +44,7 @@ class AlterHoodieTableRenameCommand(
|
||||
.setConf(hadoopConf).build()
|
||||
// Init table with new name.
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(metaClient.getTableConfig.getProperties)
|
||||
.fromProperties(metaClient.getTableConfig.getProps)
|
||||
.setTableName(newName.table)
|
||||
.initTable(hadoopConf, path)
|
||||
// Call AlterTableRenameCommand#run to rename table in meta.
|
||||
|
||||
@@ -70,13 +70,13 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
|
||||
withSparkConf(sparkSession, targetTable.storage.properties) {
|
||||
Map(
|
||||
"path" -> path.toString,
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
TABLE_NAME -> tableId.table,
|
||||
OPERATION_OPT_KEY -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_SUPPORT_TIMESTAMP -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM -> "200",
|
||||
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
TABLE_NAME.key -> tableId.table,
|
||||
OPERATION_OPT_KEY.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
@@ -189,7 +189,7 @@ object InsertIntoHoodieTableCommand {
|
||||
}
|
||||
val parameters = HoodieOptionConfig.mappingSqlOptionToHoodieParam(table.storage.properties)
|
||||
|
||||
val tableType = parameters.getOrElse(TABLE_TYPE_OPT_KEY, DEFAULT_TABLE_TYPE_OPT_VAL)
|
||||
val tableType = parameters.getOrElse(TABLE_TYPE_OPT_KEY.key, TABLE_TYPE_OPT_KEY.defaultValue)
|
||||
|
||||
val partitionFields = table.partitionColumnNames.mkString(",")
|
||||
val path = getTableLocation(table, sparkSession)
|
||||
@@ -206,8 +206,8 @@ object InsertIntoHoodieTableCommand {
|
||||
}
|
||||
|
||||
val dropDuplicate = sparkSession.conf
|
||||
.getOption(INSERT_DROP_DUPS_OPT_KEY)
|
||||
.getOrElse(DEFAULT_INSERT_DROP_DUPS_OPT_VAL)
|
||||
.getOption(INSERT_DROP_DUPS_OPT_KEY.key)
|
||||
.getOrElse(INSERT_DROP_DUPS_OPT_KEY.defaultValue)
|
||||
.toBoolean
|
||||
|
||||
val operation = if (isOverwrite) {
|
||||
@@ -236,25 +236,25 @@ object InsertIntoHoodieTableCommand {
|
||||
withSparkConf(sparkSession, options) {
|
||||
Map(
|
||||
"path" -> path,
|
||||
TABLE_TYPE_OPT_KEY -> tableType,
|
||||
TABLE_NAME -> table.identifier.table,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> tableSchema.fields.last.name,
|
||||
OPERATION_OPT_KEY -> operation,
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> keyGenClass,
|
||||
RECORDKEY_FIELD_OPT_KEY -> primaryColumns.mkString(","),
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> partitionFields,
|
||||
PAYLOAD_CLASS_OPT_KEY -> payloadClassName,
|
||||
META_SYNC_ENABLED_OPT_KEY -> "true",
|
||||
HIVE_USE_JDBC_OPT_KEY -> "false",
|
||||
HIVE_DATABASE_OPT_KEY -> table.identifier.database.getOrElse("default"),
|
||||
HIVE_TABLE_OPT_KEY -> table.identifier.table,
|
||||
HIVE_SUPPORT_TIMESTAMP -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> partitionFields,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY -> "true",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "200",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
|
||||
TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
TABLE_NAME.key -> table.identifier.table,
|
||||
PRECOMBINE_FIELD_OPT_KEY.key -> tableSchema.fields.last.name,
|
||||
OPERATION_OPT_KEY.key -> operation,
|
||||
KEYGENERATOR_CLASS_OPT_KEY.key -> keyGenClass,
|
||||
RECORDKEY_FIELD_OPT_KEY.key -> primaryColumns.mkString(","),
|
||||
PARTITIONPATH_FIELD_OPT_KEY.key -> partitionFields,
|
||||
PAYLOAD_CLASS_OPT_KEY.key -> payloadClassName,
|
||||
META_SYNC_ENABLED_OPT_KEY.key -> "true",
|
||||
HIVE_USE_JDBC_OPT_KEY.key -> "false",
|
||||
HIVE_DATABASE_OPT_KEY.key -> table.identifier.database.getOrElse("default"),
|
||||
HIVE_TABLE_OPT_KEY.key -> table.identifier.table,
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY.key -> partitionFields,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
@@ -241,9 +241,9 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
|
||||
// may be different from the target table, because the are transform logical in the update or
|
||||
// insert actions.
|
||||
var writeParams = parameters +
|
||||
(OPERATION_OPT_KEY -> UPSERT_OPERATION_OPT_VAL) +
|
||||
(HoodieWriteConfig.WRITE_SCHEMA_PROP -> getTableSchema.toString) +
|
||||
(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> targetTableType)
|
||||
(OPERATION_OPT_KEY.key -> UPSERT_OPERATION_OPT_VAL) +
|
||||
(HoodieWriteConfig.WRITE_SCHEMA_PROP.key -> getTableSchema.toString) +
|
||||
(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> targetTableType)
|
||||
|
||||
// Map of Condition -> Assignments
|
||||
val updateConditionToAssignments =
|
||||
@@ -288,8 +288,8 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
|
||||
checkInsertAssignments(insertActions)
|
||||
|
||||
var writeParams = parameters +
|
||||
(OPERATION_OPT_KEY -> INSERT_OPERATION_OPT_VAL) +
|
||||
(HoodieWriteConfig.WRITE_SCHEMA_PROP -> getTableSchema.toString)
|
||||
(OPERATION_OPT_KEY.key -> INSERT_OPERATION_OPT_VAL) +
|
||||
(HoodieWriteConfig.WRITE_SCHEMA_PROP.key -> getTableSchema.toString)
|
||||
|
||||
writeParams += (PAYLOAD_INSERT_CONDITION_AND_ASSIGNMENTS ->
|
||||
serializedInsertConditionAndExpressions(insertActions))
|
||||
@@ -431,24 +431,24 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
|
||||
withSparkConf(sparkSession, options) {
|
||||
Map(
|
||||
"path" -> path,
|
||||
RECORDKEY_FIELD_OPT_KEY -> targetKey2SourceExpression.keySet.mkString(","),
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
|
||||
TABLE_NAME -> targetTableName,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
|
||||
PAYLOAD_CLASS_OPT_KEY -> classOf[ExpressionPayload].getCanonicalName,
|
||||
META_SYNC_ENABLED_OPT_KEY -> "true",
|
||||
HIVE_USE_JDBC_OPT_KEY -> "false",
|
||||
HIVE_DATABASE_OPT_KEY -> targetTableDb,
|
||||
HIVE_TABLE_OPT_KEY -> targetTableName,
|
||||
HIVE_SUPPORT_TIMESTAMP -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY -> "true", // enable the url decode for sql.
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "200", // set the default parallelism to 200 for sql
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM -> "200",
|
||||
RECORDKEY_FIELD_OPT_KEY.key -> targetKey2SourceExpression.keySet.mkString(","),
|
||||
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD_OPT_KEY.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
|
||||
TABLE_NAME.key -> targetTableName,
|
||||
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
PAYLOAD_CLASS_OPT_KEY.key -> classOf[ExpressionPayload].getCanonicalName,
|
||||
META_SYNC_ENABLED_OPT_KEY.key -> "true",
|
||||
HIVE_USE_JDBC_OPT_KEY.key -> "false",
|
||||
HIVE_DATABASE_OPT_KEY.key -> targetTableDb,
|
||||
HIVE_TABLE_OPT_KEY.key -> targetTableName,
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true", // enable the url decode for sql.
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "200", // set the default parallelism to 200 for sql
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
})
|
||||
|
||||
@@ -46,7 +46,7 @@ class TruncateHoodieTableCommand(
|
||||
// Create MetaClient
|
||||
val metaClient = HoodieTableMetaClient.builder().setBasePath(path)
|
||||
.setConf(hadoopConf).build()
|
||||
Some(metaClient.getTableConfig.getProperties)
|
||||
Some(metaClient.getTableConfig.getProps)
|
||||
} else {
|
||||
None
|
||||
}
|
||||
|
||||
@@ -96,22 +96,22 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
|
||||
withSparkConf(sparkSession, targetTable.storage.properties) {
|
||||
Map(
|
||||
"path" -> path.toString,
|
||||
RECORDKEY_FIELD_OPT_KEY -> primaryColumns.mkString(","),
|
||||
KEYGENERATOR_CLASS_OPT_KEY -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD_OPT_KEY -> primaryColumns.head, //set the default preCombine field.
|
||||
TABLE_NAME -> tableId.table,
|
||||
OPERATION_OPT_KEY -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
|
||||
META_SYNC_ENABLED_OPT_KEY -> "false", // TODO make the meta sync enable by default.
|
||||
HIVE_USE_JDBC_OPT_KEY -> "false",
|
||||
HIVE_DATABASE_OPT_KEY -> tableId.database.getOrElse("default"),
|
||||
HIVE_TABLE_OPT_KEY -> tableId.table,
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY -> "true",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "200",
|
||||
RECORDKEY_FIELD_OPT_KEY.key -> primaryColumns.mkString(","),
|
||||
KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD_OPT_KEY.key -> primaryColumns.head, //set the default preCombine field.
|
||||
TABLE_NAME.key -> tableId.table,
|
||||
OPERATION_OPT_KEY.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
META_SYNC_ENABLED_OPT_KEY.key -> "false", // TODO make the meta sync enable by default.
|
||||
HIVE_USE_JDBC_OPT_KEY.key -> "false",
|
||||
HIVE_DATABASE_OPT_KEY.key -> tableId.database.getOrElse("default"),
|
||||
HIVE_TABLE_OPT_KEY.key -> tableId.table,
|
||||
HIVE_PARTITION_FIELDS_OPT_KEY.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING_OPT_KEY.key -> "true",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
@@ -171,7 +171,7 @@ class ExpressionPayload(record: GenericRecord,
|
||||
}
|
||||
|
||||
private def isMORTable(properties: Properties): Boolean = {
|
||||
properties.getProperty(TABLE_TYPE_OPT_KEY, null) == MOR_TABLE_TYPE_OPT_VAL
|
||||
properties.getProperty(TABLE_TYPE_OPT_KEY.key, null) == MOR_TABLE_TYPE_OPT_VAL
|
||||
}
|
||||
|
||||
private def convertToRecord(values: Array[AnyRef], schema: Schema): IndexedRecord = {
|
||||
@@ -188,9 +188,9 @@ class ExpressionPayload(record: GenericRecord,
|
||||
*/
|
||||
private def initWriteSchemaIfNeed(properties: Properties): Unit = {
|
||||
if (writeSchema == null) {
|
||||
ValidationUtils.checkArgument(properties.containsKey(HoodieWriteConfig.WRITE_SCHEMA_PROP),
|
||||
s"Missing ${HoodieWriteConfig.WRITE_SCHEMA_PROP}")
|
||||
writeSchema = new Schema.Parser().parse(properties.getProperty(HoodieWriteConfig.WRITE_SCHEMA_PROP))
|
||||
ValidationUtils.checkArgument(properties.containsKey(HoodieWriteConfig.WRITE_SCHEMA_PROP.key),
|
||||
s"Missing ${HoodieWriteConfig.WRITE_SCHEMA_PROP.key}")
|
||||
writeSchema = new Schema.Parser().parse(properties.getProperty(HoodieWriteConfig.WRITE_SCHEMA_PROP.key))
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -154,8 +154,8 @@ class HoodieStreamSource(
|
||||
} else {
|
||||
// Consume the data between (startCommitTime, endCommitTime]
|
||||
val incParams = parameters ++ Map(
|
||||
DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY -> startCommitTime(startOffset),
|
||||
DataSourceReadOptions.END_INSTANTTIME_OPT_KEY -> endOffset.commitTime
|
||||
DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key -> startCommitTime(startOffset),
|
||||
DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key -> endOffset.commitTime
|
||||
)
|
||||
|
||||
val rdd = tableType match {
|
||||
|
||||
@@ -140,22 +140,22 @@ public class HoodieJavaApp {
|
||||
// full list in HoodieWriteConfig & its package
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
// Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType)
|
||||
// insert
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
// This is the record key
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
|
||||
// this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
|
||||
// use to combine duplicate records in input/with disk val
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
|
||||
// Used by hive sync and queries
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName)
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
|
||||
// Add Key Extractor
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
// This will remove any existing data at path below, and create a
|
||||
.mode(SaveMode.Overwrite);
|
||||
|
||||
@@ -174,16 +174,16 @@ 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.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")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer.save(tablePath);
|
||||
@@ -200,17 +200,17 @@ public class HoodieJavaApp {
|
||||
writer = inputDF3.write().format("org.apache.hudi").option("hoodie.insert.shuffle.parallelism", "2")
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
.option("hoodie.delete.shuffle.parallelism", "2")
|
||||
.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")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType) // Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), "delete")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "_row_key")
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).mode(SaveMode.Append);
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "false")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
writer.save(tablePath);
|
||||
@@ -234,9 +234,9 @@ public class HoodieJavaApp {
|
||||
* Consume incrementally, only changes in commit 2 above. Currently only supported for COPY_ON_WRITE TABLE
|
||||
*/
|
||||
Dataset<Row> incQueryDF = spark.read().format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY().key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY().key(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
.load(tablePath);
|
||||
|
||||
@@ -251,23 +251,23 @@ public class HoodieJavaApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY().key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "");
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
|
||||
@@ -124,23 +124,23 @@ public class HoodieJavaGenerateApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY().key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "");
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
@@ -165,19 +165,19 @@ public class HoodieJavaGenerateApp {
|
||||
// full list in HoodieWriteConfig & its package
|
||||
.option("hoodie.upsert.shuffle.parallelism", "2")
|
||||
// Hoodie Table Type
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY(), tableType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType)
|
||||
// insert
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL())
|
||||
// This is the record key
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
|
||||
// this is the partition to place it into
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
|
||||
// use to combine duplicate records in input/with disk val
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
|
||||
// Used by hive sync and queries
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName)
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
|
||||
// Add Key Extractor
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(),
|
||||
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
|
||||
: SimpleKeyGenerator.class.getCanonicalName())
|
||||
.mode(commitType);
|
||||
|
||||
@@ -332,9 +332,9 @@ 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("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY().key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL())
|
||||
// Only changes in write 2 above
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY(), commitInstantTime1)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY().key(), commitInstantTime1)
|
||||
// For incremental view, pass in the root/base path of dataset
|
||||
.load(tablePath);
|
||||
|
||||
@@ -355,14 +355,14 @@ 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("hoodie.delete.shuffle.parallelism", "2")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY(), operationType)
|
||||
.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")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME, tableName).option("checkpointLocation", checkpointLocation)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY().key(), operationType)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY().key(), tableType)
|
||||
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key")
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition")
|
||||
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY().key(), "timestamp")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key(), "1")
|
||||
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY().key(), "true")
|
||||
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
|
||||
.outputMode(OutputMode.Append());
|
||||
|
||||
updateHiveSyncConfig(writer);
|
||||
@@ -379,18 +379,18 @@ public class HoodieJavaStreamingApp {
|
||||
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY(), "true");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL_OPT_KEY().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER_OPT_KEY().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS_OPT_KEY().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED_OPT_KEY().key(), "true");
|
||||
if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY().key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), "dateStr");
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY().key(), "dateStr");
|
||||
}
|
||||
}
|
||||
return writer;
|
||||
|
||||
@@ -94,18 +94,18 @@ public class TestHoodieDatasetBulkInsertHelper extends HoodieClientTestBase {
|
||||
private Map<String, String> getProps(boolean setAll, boolean setKeyGen, boolean setRecordKey, boolean setPartitionPath) {
|
||||
Map<String, String> props = new HashMap<>();
|
||||
if (setAll) {
|
||||
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
|
||||
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition");
|
||||
} else {
|
||||
if (setKeyGen) {
|
||||
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
|
||||
}
|
||||
if (setRecordKey) {
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
props.put(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
|
||||
}
|
||||
if (setPartitionPath) {
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition");
|
||||
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "partition");
|
||||
}
|
||||
}
|
||||
return props;
|
||||
|
||||
@@ -371,7 +371,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||
assertEquals(totalRecords, records.size());
|
||||
@@ -390,7 +390,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, false, new ArrayList<>());
|
||||
assertEquals(totalRecords, records.size());
|
||||
@@ -407,7 +407,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
|
||||
true, HoodieRecord.HOODIE_META_COLUMNS);
|
||||
@@ -425,7 +425,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
HoodieRecord.HOODIE_META_COLUMNS);
|
||||
@@ -441,7 +441,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
Arrays.asList("_row_key"));
|
||||
@@ -459,7 +459,7 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
|
||||
jsc.hadoopConfiguration(),
|
||||
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_VALIDATE, false).stream()
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
|
||||
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
|
||||
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
|
||||
Arrays.asList("_row_key"));
|
||||
@@ -550,8 +550,8 @@ public class TestBootstrap extends HoodieClientTestBase {
|
||||
HoodieWriteConfig.Builder builder = getConfigBuilder(schemaStr, IndexType.BLOOM)
|
||||
.withExternalSchemaTrasformation(true);
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key");
|
||||
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "datestr");
|
||||
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY().key(), "_row_key");
|
||||
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY().key(), "datestr");
|
||||
builder = builder.withProps(properties);
|
||||
return builder;
|
||||
}
|
||||
|
||||
@@ -53,9 +53,9 @@ class TestDataSourceDefaults {
|
||||
|
||||
private def getKeyConfig(recordKeyFieldName: String, partitionPathField: String, hiveStylePartitioning: String): TypedProperties = {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, recordKeyFieldName)
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionPathField)
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, hiveStylePartitioning)
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, recordKeyFieldName)
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, partitionPathField)
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key, hiveStylePartitioning)
|
||||
props
|
||||
}
|
||||
|
||||
@@ -73,7 +73,7 @@ class TestDataSourceDefaults {
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -84,7 +84,7 @@ class TestDataSourceDefaults {
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -96,7 +96,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key(), "partitionField")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -107,7 +107,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partitionField")
|
||||
val keyGen = new SimpleKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -179,8 +179,8 @@ class TestDataSourceDefaults {
|
||||
try {
|
||||
baseRecord.put("field1", "")
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -191,8 +191,8 @@ class TestDataSourceDefaults {
|
||||
// if record key is empty, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
@@ -206,8 +206,8 @@ class TestDataSourceDefaults {
|
||||
try {
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "name")
|
||||
new SimpleKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -218,8 +218,8 @@ class TestDataSourceDefaults {
|
||||
// if record key is null, throw error. Using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "name")
|
||||
keyGen = new SimpleKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
@@ -237,8 +237,8 @@ class TestDataSourceDefaults {
|
||||
}
|
||||
|
||||
class UserDefinedKeyGenerator(props: TypedProperties) extends KeyGenerator(props) with SparkKeyGeneratorInterface {
|
||||
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY)
|
||||
val recordKeyProp: String = props.getString(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key)
|
||||
val partitionPathProp: String = props.getString(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key)
|
||||
val STRUCT_NAME: String = "hoodieRowTopLevelField"
|
||||
val NAMESPACE: String = "hoodieRow"
|
||||
var converterFn: Function1[Any, Any] = _
|
||||
@@ -275,7 +275,7 @@ class TestDataSourceDefaults {
|
||||
// partition path field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -286,7 +286,7 @@ class TestDataSourceDefaults {
|
||||
// partition path field not specified using Row
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -298,7 +298,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partitionField")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -309,7 +309,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partitionField")
|
||||
val keyGen = new ComplexKeyGenerator(props)
|
||||
keyGen.getPartitionPath(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -391,8 +391,8 @@ class TestDataSourceDefaults {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "field1,name")
|
||||
new ComplexKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -405,8 +405,8 @@ class TestDataSourceDefaults {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "field1,name")
|
||||
keyGen = new ComplexKeyGenerator(props)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
@@ -449,7 +449,7 @@ class TestDataSourceDefaults {
|
||||
|
||||
// top level, partition value not included
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1,name")
|
||||
keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
val hk2 = keyGen.getKey(baseRecord)
|
||||
assertEquals("field1:field1,name:name1", hk2.getRecordKey)
|
||||
@@ -483,7 +483,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partitionField")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -494,7 +494,7 @@ class TestDataSourceDefaults {
|
||||
// recordkey field not specified
|
||||
try {
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partitionField")
|
||||
props.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partitionField")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -528,7 +528,7 @@ class TestDataSourceDefaults {
|
||||
baseRecord.put("name", "")
|
||||
baseRecord.put("field1", null)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1,name")
|
||||
new GlobalDeleteKeyGenerator(props).getKey(baseRecord)
|
||||
fail("Should have errored out")
|
||||
} catch {
|
||||
@@ -542,7 +542,7 @@ class TestDataSourceDefaults {
|
||||
baseRecord.put("field1", null)
|
||||
baseRow = KeyGeneratorTestUtilities.getRow(baseRecord, schema, structType)
|
||||
val props = new TypedProperties()
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY, "field1,name")
|
||||
props.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key, "field1,name")
|
||||
val keyGen = new GlobalDeleteKeyGenerator(props)
|
||||
keyGen.getRecordKey(baseRow)
|
||||
fail("Should have errored out")
|
||||
@@ -573,7 +573,7 @@ class TestDataSourceDefaults {
|
||||
val baseOrderingVal: Object = baseRecord.get("favoriteIntNumber")
|
||||
val fieldSchema: Schema = baseRecord.getSchema().getField("favoriteIntNumber").schema()
|
||||
val props = new TypedProperties()
|
||||
props.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP, "favoriteIntNumber");
|
||||
props.put(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY, "favoriteIntNumber");
|
||||
|
||||
val basePayload = new OverwriteWithLatestAvroPayload(baseRecord, HoodieAvroUtils.convertValueForSpecificDataTypes(fieldSchema, baseOrderingVal).asInstanceOf[Comparable[_]])
|
||||
|
||||
|
||||
@@ -47,10 +47,10 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
@@ -69,8 +69,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
@@ -88,8 +88,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, keyGenerator)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key, keyGenerator)
|
||||
.option(Config.TIMESTAMP_TYPE_FIELD_PROP, TimestampType.DATE_STRING.name())
|
||||
.option(Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, "yyyy/MM/dd")
|
||||
.option(Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, "yyyy-MM-dd")
|
||||
@@ -109,9 +109,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, keyGenerator)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "partition:simple")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key, keyGenerator)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "partition:simple")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
@@ -128,8 +128,8 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
@@ -166,13 +166,13 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "false")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, useMetaFileList)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY.key, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY.key, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY.key, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, "false")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
metaClient = HoodieTableMetaClient.reload(metaClient)
|
||||
@@ -200,12 +200,12 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
|
||||
s"2021/03/0${i % 2 + 1}", "10")).toDF("id", "name", "price", "version", "dt", "hh")
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, "false")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY.key, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY.key, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY.key, "dt,hh")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY.key, classOf[ComplexKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, "false")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
|
||||
@@ -20,10 +20,12 @@ package org.apache.hudi.functional
|
||||
import java.time.Instant
|
||||
import java.util
|
||||
import java.util.{Collections, Date, UUID}
|
||||
|
||||
import org.apache.commons.io.FileUtils
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap}
|
||||
import org.apache.hudi.common.config.HoodieConfig
|
||||
import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload}
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
|
||||
@@ -54,13 +56,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val originals = HoodieWriterUtils.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, TABLE_TYPE_OPT_KEY -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modifier = Map(OPERATION_OPT_KEY.key -> INSERT_OPERATION_OPT_VAL, TABLE_TYPE_OPT_KEY.key -> MOR_TABLE_TYPE_OPT_VAL, rhsKey -> rhsVal)
|
||||
val modified = HoodieWriterUtils.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 (TABLE_TYPE_OPT_KEY, _) => matcher(TABLE_TYPE_OPT_KEY, MOR_TABLE_TYPE_OPT_VAL)
|
||||
case ("hoodie.datasource.write.operation", _) => matcher("hoodie.datasource.write.operation", INSERT_OPERATION_OPT_VAL)
|
||||
case ("hoodie.datasource.write.table.type", _) => matcher("hoodie.datasource.write.table.type", MOR_TABLE_TYPE_OPT_VAL)
|
||||
case (`rhsKey`, _) => matcher(rhsKey, rhsVal)
|
||||
case (k, v) => matcher(k, v)
|
||||
}
|
||||
@@ -70,7 +72,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val session = SparkSession.builder().appName("hoodie_test").master("local").getOrCreate()
|
||||
try {
|
||||
val sqlContext = session.sqlContext
|
||||
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TABLE_NAME -> "hoodie_test_tbl")
|
||||
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test_tbl")
|
||||
val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options,
|
||||
session.emptyDataFrame))
|
||||
assert(e.getMessage.contains("spark.serializer"))
|
||||
@@ -90,7 +92,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
@@ -99,7 +101,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//on same path try append with different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val barTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_bar_tbl",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_bar_tbl",
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4")
|
||||
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
|
||||
@@ -108,7 +110,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
|
||||
//on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception
|
||||
val deleteTableParams = barTableParams ++ Map(OPERATION_OPT_KEY -> "delete")
|
||||
val deleteTableParams = barTableParams ++ Map(OPERATION_OPT_KEY.key -> "delete")
|
||||
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableParams, dataFrame2))
|
||||
assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
|
||||
} finally {
|
||||
@@ -128,14 +130,14 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -181,15 +183,15 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
|
||||
INSERT_DROP_DUPS_OPT_KEY -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
INSERT_DROP_DUPS_OPT_KEY.key -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -220,13 +222,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY -> "false",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY.key -> "false",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -236,7 +238,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val recordsSeq = convertRowListToSeq(records)
|
||||
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
|
||||
// write to Hudi
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY, df)
|
||||
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key, df)
|
||||
|
||||
// collect all parition paths to issue read of parquet files
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
@@ -271,13 +273,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.ENABLE_ROW_WRITER_OPT_KEY.key -> "true",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
|
||||
@@ -329,13 +331,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> classOf[SimpleKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -407,14 +409,14 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
.save(srcPath.toAbsolutePath.toString)
|
||||
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key -> srcPath.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key -> "4",
|
||||
DataSourceWriteOptions.OPERATION_OPT_KEY.key -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
val client = spy(DataSourceUtils.createHoodieClient(
|
||||
@@ -450,13 +452,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val hoodieFooTableName = "hoodie_foo_tbl_" + tableType
|
||||
//create a new table
|
||||
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
|
||||
HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName,
|
||||
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
|
||||
"hoodie.insert.shuffle.parallelism" -> "1",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key -> tableType,
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
|
||||
|
||||
// generate the inserts
|
||||
@@ -530,7 +532,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
initSparkContext("test build sync config")
|
||||
val addSqlTablePropertiesMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("addSqlTableProperties",
|
||||
classOf[SQLConf], classOf[StructType], classOf[Map[_, _]])
|
||||
classOf[SQLConf], classOf[StructType], classOf[HoodieConfig])
|
||||
addSqlTablePropertiesMethod.setAccessible(true)
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
@@ -538,22 +540,23 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val basePath = "/tmp/hoodie_test"
|
||||
val params = Map(
|
||||
"path" -> basePath,
|
||||
DataSourceWriteOptions.TABLE_NAME_OPT_KEY -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX -> "true"
|
||||
DataSourceWriteOptions.TABLE_NAME_OPT_KEY.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.key -> "true"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val newParams = addSqlTablePropertiesMethod.invoke(HoodieSparkSqlWriter,
|
||||
spark.sessionState.conf, structType, parameters)
|
||||
.asInstanceOf[Map[String, String]]
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val newHoodieConfig = addSqlTablePropertiesMethod.invoke(HoodieSparkSqlWriter,
|
||||
spark.sessionState.conf, structType, hoodieConfig)
|
||||
.asInstanceOf[HoodieConfig]
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[Map[_, _]])
|
||||
classOf[HoodieConfig])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(basePath), newParams).asInstanceOf[HiveSyncConfig]
|
||||
new Path(basePath), newHoodieConfig).asInstanceOf[HiveSyncConfig]
|
||||
|
||||
assertTrue(hiveSyncConfig.skipROSuffix)
|
||||
assertResult("spark.sql.sources.provider=hudi\n" +
|
||||
@@ -580,7 +583,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
initSparkContext("test build sync config for skip Ro suffix vals")
|
||||
val addSqlTablePropertiesMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("addSqlTableProperties",
|
||||
classOf[SQLConf], classOf[StructType], classOf[Map[_, _]])
|
||||
classOf[SQLConf], classOf[StructType], classOf[HoodieConfig])
|
||||
addSqlTablePropertiesMethod.setAccessible(true)
|
||||
|
||||
val schema = DataSourceTestUtils.getStructTypeExampleSchema
|
||||
@@ -588,21 +591,22 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
|
||||
val basePath = "/tmp/hoodie_test"
|
||||
val params = Map(
|
||||
"path" -> basePath,
|
||||
DataSourceWriteOptions.TABLE_NAME_OPT_KEY -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY -> "partition"
|
||||
DataSourceWriteOptions.TABLE_NAME_OPT_KEY.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY.key -> "partition"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val newParams = addSqlTablePropertiesMethod.invoke(HoodieSparkSqlWriter,
|
||||
spark.sessionState.conf, structType, parameters)
|
||||
.asInstanceOf[Map[String, String]]
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val newHoodieConfig = addSqlTablePropertiesMethod.invoke(HoodieSparkSqlWriter,
|
||||
spark.sessionState.conf, structType, hoodieConfig)
|
||||
.asInstanceOf[HoodieConfig]
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[Map[_, _]])
|
||||
classOf[HoodieConfig])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(basePath), newParams).asInstanceOf[HiveSyncConfig]
|
||||
new Path(basePath), newHoodieConfig).asInstanceOf[HiveSyncConfig]
|
||||
|
||||
assertFalse(hiveSyncConfig.skipROSuffix)
|
||||
}
|
||||
|
||||
@@ -55,10 +55,10 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
"hoodie.bulkinsert.shuffle.parallelism" -> "2",
|
||||
"hoodie.delete.shuffle.parallelism" -> "1",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
val verificationCol: String = "driver"
|
||||
@@ -84,7 +84,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF = spark.read.json(spark.sparkContext.parallelize(records, 2))
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -112,14 +112,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.option("hoodie.keep.max.commits", "2")
|
||||
.option("hoodie.cleaner.commits.retained", "0")
|
||||
.option("hoodie.datasource.write.row.writer.enable", "true")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
||||
.mode(if (i == 0) SaveMode.Overwrite else SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.builder().setConf(spark.sparkContext.hadoopConfiguration).setBasePath(basePath).build()
|
||||
val actualSchema = new TableSchemaResolver(tableMetaClient).getTableAvroSchemaWithoutMetadataFields
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TABLE_NAME))
|
||||
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TABLE_NAME.key))
|
||||
spark.sparkContext.getConf.registerKryoClasses(
|
||||
Array(classOf[org.apache.avro.generic.GenericData],
|
||||
classOf[org.apache.avro.Schema]))
|
||||
@@ -149,7 +149,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -169,8 +169,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -179,7 +179,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Snapshot query
|
||||
val snapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF1.count())
|
||||
|
||||
@@ -204,7 +204,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -213,7 +213,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Snapshot Query
|
||||
val snapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, snapshotDF3.count()) // still 100, since we only updated
|
||||
|
||||
@@ -221,9 +221,9 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
// 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.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.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, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, hoodieIncViewDF1.count()) // 100 initial inserts must be pulled
|
||||
var countsPerCommit = hoodieIncViewDF1.groupBy("_hoodie_commit_time").count().collect()
|
||||
@@ -235,14 +235,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val emptyDF = spark.read.json(spark.sparkContext.parallelize(emptyRecords, 1))
|
||||
emptyDF.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP, isMetadataEnabled)
|
||||
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime2)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
@@ -252,16 +252,16 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// pull the latest commit within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime2)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, "/2016/*/*/*")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime2)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key, "/2016/*/*/*")
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2016")).count(), hoodieIncViewDF3.count())
|
||||
|
||||
val timeTravelDF = 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, firstCommit)
|
||||
.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, firstCommit)
|
||||
.load(basePath)
|
||||
assertEquals(100, timeTravelDF.count()) // 100 initial inserts must be pulled
|
||||
}
|
||||
@@ -271,7 +271,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -279,7 +279,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -297,7 +297,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -305,7 +305,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -324,7 +324,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -333,7 +333,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -342,7 +342,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2))
|
||||
inputDF3.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OVERWRITE_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -380,7 +380,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -389,7 +389,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -439,7 +439,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF1 = spark.read.format("org.apache.hudi")
|
||||
@@ -451,7 +451,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(records2, 2))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY, "true")
|
||||
.option(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY.key, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hoodieROViewDF2 = spark.read.format("org.apache.hudi")
|
||||
@@ -459,8 +459,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assertEquals(hoodieROViewDF2.count(), totalUniqueKeyToGenerate)
|
||||
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(hoodieIncViewDF2.count(), insert2NewKeyCnt)
|
||||
}
|
||||
@@ -502,8 +502,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, "true")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP.key, "true")
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -516,7 +516,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, keyGenerator)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key, keyGenerator)
|
||||
.mode(SaveMode.Overwrite)
|
||||
}
|
||||
|
||||
@@ -666,8 +666,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
@@ -692,14 +692,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(N + 1, hoodieIncViewDF1.count())
|
||||
}
|
||||
|
||||
@@ -41,16 +41,16 @@ class TestDataSourceForBootstrap {
|
||||
|
||||
var spark: SparkSession = _
|
||||
val commonOpts: Map[String, String] = Map(
|
||||
HoodieWriteConfig.INSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM -> "4",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.BULKINSERT_PARALLELISM.key -> "4",
|
||||
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM.key -> "4",
|
||||
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
)
|
||||
var basePath: String = _
|
||||
var srcPath: String = _
|
||||
@@ -117,9 +117,9 @@ class TestDataSourceForBootstrap {
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -168,11 +168,11 @@ class TestDataSourceForBootstrap {
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
// Required because source data is hive style partitioned
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, "true")
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key, "true")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -221,9 +221,9 @@ class TestDataSourceForBootstrap {
|
||||
updateDf1.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -241,9 +241,9 @@ class TestDataSourceForBootstrap {
|
||||
updateDF2.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -282,7 +282,7 @@ class TestDataSourceForBootstrap {
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
@@ -295,11 +295,11 @@ class TestDataSourceForBootstrap {
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, "1")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_PROP.key, "true")
|
||||
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS_PROP.key, "1")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -309,14 +309,14 @@ class TestDataSourceForBootstrap {
|
||||
// Read table after upsert and verify count. Since we have inline compaction enabled the RO view will have
|
||||
// the updated rows.
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF2.count())
|
||||
assertEquals(numRecordsUpdate, hoodieROViewDF2.filter(s"timestamp == $updateTimestamp").count())
|
||||
// Test query without "*" for MOR READ_OPTIMIZED
|
||||
val hoodieROViewDFWithBasePath = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDFWithBasePath.count())
|
||||
@@ -347,13 +347,13 @@ class TestDataSourceForBootstrap {
|
||||
|
||||
// Read bootstrapped table and verify count
|
||||
val hoodieROViewDF1 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF1.count())
|
||||
// Read bootstrapped table without "*"
|
||||
val hoodieROViewDFWithBasePath = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(numRecords, hoodieROViewDFWithBasePath.count())
|
||||
@@ -363,16 +363,16 @@ class TestDataSourceForBootstrap {
|
||||
updateDf1.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
// Read table after upsert and verify the value
|
||||
assertEquals(1, HoodieDataSourceHelpers.listCommitsSince(fs, basePath, commitInstantTime1).size())
|
||||
val hoodieROViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
hoodieROViewDF2.collect()
|
||||
@@ -386,9 +386,9 @@ class TestDataSourceForBootstrap {
|
||||
updateDF2.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -398,7 +398,7 @@ class TestDataSourceForBootstrap {
|
||||
// Read table after upsert and verify count. Since we have inline compaction off the RO view will have
|
||||
// no updated rows.
|
||||
val hoodieROViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY,
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key,
|
||||
DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*")
|
||||
assertEquals(numRecords, hoodieROViewDF3.count())
|
||||
@@ -428,12 +428,12 @@ class TestDataSourceForBootstrap {
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[SimpleKeyGenerator].getName)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_MODE_SELECTOR.key, classOf[FullRecordBootstrapModeSelector].getName)
|
||||
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER.key, classOf[SparkParquetBootstrapDataProvider].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -455,9 +455,9 @@ class TestDataSourceForBootstrap {
|
||||
updateDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, "datestr")
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, "datestr")
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
@@ -478,11 +478,11 @@ class TestDataSourceForBootstrap {
|
||||
bootstrapDF.write
|
||||
.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, tableType)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY, partitionColumns.getOrElse(""))
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS, classOf[SimpleKeyGenerator].getName)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, tableType)
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key, partitionColumns.getOrElse(""))
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP.key, srcPath)
|
||||
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[SimpleKeyGenerator].getName)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -496,9 +496,9 @@ class TestDataSourceForBootstrap {
|
||||
// incrementally pull only changes in the bootstrap commit, which would pull all the initial records written
|
||||
// during bootstrap
|
||||
val hoodieIncViewDF1 = spark.read.format("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, bootstrapCommitInstantTime)
|
||||
.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, bootstrapCommitInstantTime)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(numRecords, hoodieIncViewDF1.count())
|
||||
@@ -509,8 +509,8 @@ class TestDataSourceForBootstrap {
|
||||
// incrementally pull only changes after bootstrap commit, which would pull only the updated records in the
|
||||
// later commits
|
||||
val hoodieIncViewDF2 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, bootstrapCommitInstantTime)
|
||||
.load(basePath);
|
||||
|
||||
assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
|
||||
@@ -522,9 +522,9 @@ class TestDataSourceForBootstrap {
|
||||
val relativePartitionPath = if (isHiveStylePartitioned) "/datestr=2020-04-02/*" else "/2020-04-02/*"
|
||||
// pull the update commits within certain partitions
|
||||
val hoodieIncViewDF3 = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, bootstrapCommitInstantTime)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY, relativePartitionPath)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, bootstrapCommitInstantTime)
|
||||
.option(DataSourceReadOptions.INCR_PATH_GLOB_OPT_KEY.key, relativePartitionPath)
|
||||
.load(basePath)
|
||||
|
||||
assertEquals(hoodieIncViewDF2.filter(col("_hoodie_partition_path").contains("2020-04-02")).count(),
|
||||
|
||||
@@ -47,10 +47,10 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
val verificationCol: String = "driver"
|
||||
@@ -79,8 +79,8 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.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.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
@@ -88,7 +88,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Read RO View
|
||||
val hudiRODF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
|
||||
val insertCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
@@ -106,7 +106,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
// Read Snapshot query
|
||||
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
|
||||
assertEquals(List(updateCommitTime), updateCommitTimes)
|
||||
@@ -134,13 +134,13 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.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.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
@@ -154,7 +154,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count()) // still 100, since we only updated
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
@@ -166,9 +166,9 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
// 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)
|
||||
.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, commit1Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF1.count())
|
||||
assertEquals(1, hudiIncDF1.select("_hoodie_commit_time").distinct().count())
|
||||
@@ -176,9 +176,9 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commit1Time)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF2.count())
|
||||
assertEquals(1, hudiIncDF2.select("_hoodie_commit_time").distinct().count())
|
||||
@@ -187,9 +187,9 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// 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)
|
||||
.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, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(100, hudiIncDF3.count())
|
||||
// log file being load
|
||||
@@ -198,8 +198,8 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Unmerge
|
||||
val hudiSnapshotSkipMergeDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotSkipMergeDF2.count())
|
||||
assertEquals(100, hudiSnapshotSkipMergeDF2.select("_hoodie_record_key").distinct().count())
|
||||
@@ -207,7 +207,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// Test Read Optimized Query on MOR table
|
||||
val hudiRODF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiRODF2.count())
|
||||
|
||||
@@ -221,7 +221,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// still 100, because we only updated the existing records
|
||||
assertEquals(100, hudiSnapshotDF3.count())
|
||||
@@ -234,17 +234,17 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// 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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, "000")
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(200, hudiIncDF4SkipMerge.count())
|
||||
|
||||
@@ -261,7 +261,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF4 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
// 200, because we insert 100 records to a new partition
|
||||
assertEquals(200, hudiSnapshotDF4.count())
|
||||
@@ -270,8 +270,8 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// 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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(150, hudiIncDF5.count())
|
||||
|
||||
@@ -287,7 +287,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(200, hudiSnapshotDF5.count())
|
||||
|
||||
@@ -302,13 +302,13 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commit5Time)
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY.key, commit6Time)
|
||||
.load(basePath)
|
||||
// compaction updated 150 rows + inserted 2 new row
|
||||
assertEquals(152, hudiIncDF6.count())
|
||||
@@ -324,13 +324,13 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.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.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count()) // still 100, since we only updated
|
||||
|
||||
@@ -344,7 +344,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(50, hudiSnapshotDF2.count()) // 50 records were deleted
|
||||
assertEquals(hudiSnapshotDF2.select("_hoodie_commit_time").distinct().count(), 1)
|
||||
@@ -355,15 +355,15 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
// unmerge query, skip the delete records
|
||||
val hudiSnapshotDF2Unmerge = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commit2Time)
|
||||
.load(basePath)
|
||||
assertEquals(0, hudiIncDF1.count())
|
||||
|
||||
@@ -377,7 +377,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF3 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(0, hudiSnapshotDF3.count()) // 100 records were deleted, 0 record to load
|
||||
}
|
||||
@@ -392,12 +392,12 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.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.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
val commit1Time = hudiSnapshotDF1.select("_hoodie_commit_time").head().get(0).toString
|
||||
|
||||
@@ -420,20 +420,20 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
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.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.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")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.REALTIME_MERGE_OPT_KEY.key, DataSourceReadOptions.REALTIME_SKIP_MERGE_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.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)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commit1Time)
|
||||
.load(basePath)
|
||||
|
||||
// filter first commit and only read log records
|
||||
@@ -473,12 +473,12 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.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.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF1.count())
|
||||
|
||||
@@ -490,7 +490,7 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
|
||||
.load(basePath + "/*/*/*/*")
|
||||
assertEquals(100, hudiSnapshotDF2.count())
|
||||
|
||||
@@ -528,12 +528,12 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
df.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
// use DefaultHoodieRecordPayload here
|
||||
.option(PAYLOAD_CLASS_OPT_KEY, classOf[DefaultHoodieRecordPayload].getCanonicalName)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY, "")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.option(PAYLOAD_CLASS_OPT_KEY.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(RECORDKEY_FIELD_OPT_KEY.key, "id")
|
||||
.option(PRECOMBINE_FIELD_OPT_KEY.key, "version")
|
||||
.option(PARTITIONPATH_FIELD_OPT_KEY.key, "")
|
||||
.option(KEYGENERATOR_CLASS_OPT_KEY.key, classOf[NonpartitionedKeyGenerator].getName)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
}
|
||||
@@ -575,9 +575,9 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
|
||||
@@ -602,15 +602,15 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
val inputDF2 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records2), 2))
|
||||
inputDF2.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
// Incremental query without "*" in path
|
||||
val hoodieIncViewDF1 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime1)
|
||||
.load(basePath)
|
||||
assertEquals(N + 1, hoodieIncViewDF1.count())
|
||||
}
|
||||
@@ -627,10 +627,10 @@ class TestMORDataSource extends HoodieClientTestBase {
|
||||
|
||||
inputDF1.write.format("hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY, partitionEncode)
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY, hiveStylePartition)
|
||||
.option(DataSourceWriteOptions.OPERATION_OPT_KEY.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
|
||||
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key, partitionEncode)
|
||||
.option(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key, hiveStylePartition)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
|
||||
@@ -29,11 +29,11 @@ class TestStreamingSource extends StreamTest {
|
||||
|
||||
import testImplicits._
|
||||
private val commonOptions = Map(
|
||||
RECORDKEY_FIELD_OPT_KEY -> "id",
|
||||
PRECOMBINE_FIELD_OPT_KEY -> "ts",
|
||||
INSERT_PARALLELISM -> "4",
|
||||
UPSERT_PARALLELISM -> "4",
|
||||
DELETE_PARALLELISM -> "4"
|
||||
RECORDKEY_FIELD_OPT_KEY.key -> "id",
|
||||
PRECOMBINE_FIELD_OPT_KEY.key -> "ts",
|
||||
INSERT_PARALLELISM.key -> "4",
|
||||
UPSERT_PARALLELISM.key -> "4",
|
||||
DELETE_PARALLELISM.key -> "4"
|
||||
)
|
||||
private val columns = Seq("id", "name", "price", "ts")
|
||||
|
||||
@@ -47,7 +47,7 @@ class TestStreamingSource extends StreamTest {
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(COPY_ON_WRITE)
|
||||
.setTableName(getTableName(tablePath))
|
||||
.setPayloadClassName(DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY.defaultValue)
|
||||
.initTable(spark.sessionState.newHadoopConf(), tablePath)
|
||||
|
||||
addData(tablePath, Seq(("1", "a1", "10", "000")))
|
||||
@@ -97,7 +97,7 @@ class TestStreamingSource extends StreamTest {
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(MERGE_ON_READ)
|
||||
.setTableName(getTableName(tablePath))
|
||||
.setPayloadClassName(DataSourceWriteOptions.DEFAULT_PAYLOAD_OPT_VAL)
|
||||
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY.defaultValue)
|
||||
.initTable(spark.sessionState.newHadoopConf(), tablePath)
|
||||
|
||||
addData(tablePath, Seq(("1", "a1", "10", "000")))
|
||||
@@ -140,7 +140,7 @@ class TestStreamingSource extends StreamTest {
|
||||
.write
|
||||
.format("org.apache.hudi")
|
||||
.options(commonOptions)
|
||||
.option(TABLE_NAME, getTableName(inputPath))
|
||||
.option(TABLE_NAME.key, getTableName(inputPath))
|
||||
.mode(SaveMode.Append)
|
||||
.save(inputPath)
|
||||
}
|
||||
|
||||
@@ -48,10 +48,10 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
val commonOpts = Map(
|
||||
"hoodie.insert.shuffle.parallelism" -> "4",
|
||||
"hoodie.upsert.shuffle.parallelism" -> "4",
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME -> "hoodie_test"
|
||||
DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY.key -> "_row_key",
|
||||
DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY.key -> "partition",
|
||||
DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY.key -> "timestamp",
|
||||
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
|
||||
)
|
||||
|
||||
@BeforeEach override def setUp() {
|
||||
@@ -138,9 +138,9 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
// 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.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, "000")
|
||||
.option(DataSourceReadOptions.END_INSTANTTIME_OPT_KEY, firstCommit)
|
||||
.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, firstCommit)
|
||||
.load(destPath)
|
||||
assertEquals(100, hoodieIncViewDF1.count())
|
||||
// 100 initial inserts must be pulled
|
||||
@@ -150,8 +150,8 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
|
||||
// pull the latest commit
|
||||
val hoodieIncViewDF2 = spark.read.format("org.apache.hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY, commitInstantTime1)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, commitInstantTime1)
|
||||
.load(destPath)
|
||||
|
||||
assertEquals(uniqueKeyCnt, hoodieIncViewDF2.count()) // 100 records must be pulled
|
||||
@@ -191,9 +191,9 @@ class TestStructuredStreaming extends HoodieClientTestBase {
|
||||
}
|
||||
|
||||
def getInlineClusteringOpts( isInlineClustering: String, clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = {
|
||||
commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING_PROP -> isInlineClustering,
|
||||
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP -> clusteringNumCommit,
|
||||
HoodieStorageConfig.PARQUET_FILE_MAX_BYTES -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
|
||||
commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key -> isInlineClustering,
|
||||
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT_PROP.key -> clusteringNumCommit,
|
||||
HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
|
||||
)
|
||||
}
|
||||
|
||||
|
||||
@@ -496,9 +496,9 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
|
||||
)
|
||||
// Test incremental query
|
||||
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, firstCommitTime)
|
||||
.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, firstCommitTime)
|
||||
.load(targetBasePath)
|
||||
hudiIncDF1.createOrReplaceTempView("inc1")
|
||||
checkAnswer(s"select id, name, price, _ts from inc1")(
|
||||
@@ -521,8 +521,8 @@ class TestMergeIntoTable extends TestHoodieSqlBase {
|
||||
)
|
||||
// Test incremental query
|
||||
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, secondCommitTime)
|
||||
.option(DataSourceReadOptions.QUERY_TYPE_OPT_KEY.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL)
|
||||
.option(DataSourceReadOptions.BEGIN_INSTANTTIME_OPT_KEY.key, secondCommitTime)
|
||||
.load(targetBasePath)
|
||||
hudiIncDF2.createOrReplaceTempView("inc2")
|
||||
checkAnswer(s"select id, name, price, _ts from inc2 order by id")(
|
||||
|
||||
Reference in New Issue
Block a user