Restore 0.8.0 config keys with deprecated annotation (#3506)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -77,7 +77,7 @@ public class HoodieDatasetBulkInsertHelper {
|
||||
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(config.getProps());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS().key());
|
||||
String keyGeneratorClass = properties.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key());
|
||||
BuiltinKeyGenerator keyGenerator = (BuiltinKeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, properties);
|
||||
StructType structTypeForUDF = rows.schema();
|
||||
|
||||
|
||||
@@ -17,8 +17,6 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConverters._
|
||||
import org.apache.hadoop.fs.{FileStatus, Path}
|
||||
import org.apache.hudi.DataSourceReadOptions.{QUERY_TYPE, QUERY_TYPE_SNAPSHOT_OPT_VAL}
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
@@ -26,21 +24,23 @@ import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model.FileSlice
|
||||
import org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.table.view.{FileSystemViewStorageConfig, HoodieTableFileSystemView}
|
||||
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BoundReference, Expression, InterpretedPredicate}
|
||||
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
|
||||
import org.apache.spark.sql.catalyst.{InternalRow, expressions}
|
||||
import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.unsafe.types.UTF8String
|
||||
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
@@ -122,12 +122,12 @@ case class HoodieFileIndex(
|
||||
|
||||
// To support metadata listing via Spark SQL we allow users to pass the config via SQL Conf in spark session. Users
|
||||
// would be able to run SET hoodie.metadata.enable=true in the spark sql session to enable metadata listing.
|
||||
properties.put(HoodieMetadataConfig.METADATA_ENABLE_PROP,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(),
|
||||
properties.put(HoodieMetadataConfig.ENABLE,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.ENABLE.key(),
|
||||
HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS.toString))
|
||||
properties.put(HoodieMetadataConfig.METADATA_VALIDATE_PROP,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.METADATA_VALIDATE_PROP.key(),
|
||||
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue().toString))
|
||||
properties.put(HoodieMetadataConfig.VALIDATE_ENABLE,
|
||||
sqlConf.getConfString(HoodieMetadataConfig.VALIDATE_ENABLE.key(),
|
||||
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue().toString))
|
||||
properties.putAll(options.asJava)
|
||||
properties
|
||||
}
|
||||
|
||||
@@ -18,9 +18,6 @@
|
||||
package org.apache.hudi
|
||||
|
||||
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
@@ -35,7 +32,7 @@ import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, Write
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH, BOOTSTRAP_INDEX_CLASS}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
|
||||
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
|
||||
@@ -53,6 +50,8 @@ import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{DataFrame, Dataset, Row, SQLContext, SaveMode, SparkSession}
|
||||
import org.apache.spark.{SPARK_VERSION, SparkContext}
|
||||
|
||||
import java.util
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
@@ -78,7 +77,7 @@ object HoodieSparkSqlWriter {
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.get("path")
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
|
||||
val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
|
||||
asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
|
||||
asyncClusteringTriggerFnDefined = asyncClusteringTriggerFn.isDefined
|
||||
if (path.isEmpty) {
|
||||
@@ -121,10 +120,10 @@ object HoodieSparkSqlWriter {
|
||||
val partitionColumns = HoodieSparkUtils.getPartitionColumns(keyGenerator, toProperties(parameters))
|
||||
// Create the table if not present
|
||||
if (!tableExists) {
|
||||
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
|
||||
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
|
||||
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
|
||||
val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(tableType)
|
||||
@@ -132,12 +131,12 @@ object HoodieSparkSqlWriter {
|
||||
.setRecordKeyFields(recordKeyFields)
|
||||
.setBaseFileFormat(baseFileFormat)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS))
|
||||
.setPayloadClassName(hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
|
||||
.setPartitionFields(partitionColumns)
|
||||
.setPopulateMetaFields(populateMetaFields)
|
||||
.setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
|
||||
.setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS))
|
||||
.setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
|
||||
.initTable(sparkContext.hadoopConfiguration, path.get)
|
||||
tableConfig = tableMetaClient.getTableConfig
|
||||
}
|
||||
@@ -169,8 +168,8 @@ 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.key)))
|
||||
null, path.get, tblName,
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -201,7 +200,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.key)))
|
||||
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
|
||||
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
// Issue delete partitions
|
||||
client.startCommitWithTime(instantTime, commitActionType)
|
||||
@@ -235,9 +234,9 @@ object HoodieSparkSqlWriter {
|
||||
.asInstanceOf[Comparable[_]]
|
||||
DataSourceUtils.createHoodieRecord(processedRecord,
|
||||
orderingVal, keyGenerator.getKey(gr),
|
||||
hoodieConfig.getString(PAYLOAD_CLASS))
|
||||
hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
} else {
|
||||
DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS))
|
||||
DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_NAME))
|
||||
}
|
||||
hoodieRecord
|
||||
}).toJavaRDD()
|
||||
@@ -245,7 +244,7 @@ object HoodieSparkSqlWriter {
|
||||
val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
|
||||
// Create a HoodieWriteClient & issue the write.
|
||||
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path.get,
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT.key)
|
||||
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
|
||||
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
|
||||
|
||||
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
|
||||
@@ -334,12 +333,12 @@ object HoodieSparkSqlWriter {
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val path = parameters.getOrElse("path", throw new HoodieException("'path' must be set."))
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TABLE_NAME, s"'${HoodieWriteConfig.TABLE_NAME.key}' must be set.")
|
||||
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
|
||||
val tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE)
|
||||
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BOOTSTRAP_BASE_PATH,
|
||||
s"'${BOOTSTRAP_BASE_PATH.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
|
||||
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BASE_PATH,
|
||||
s"'${BASE_PATH.key}' is required for '${BOOTSTRAP_OPERATION_OPT_VAL}'" +
|
||||
" operation'")
|
||||
val bootstrapIndexClass = hoodieConfig.getStringOrDefault(BOOTSTRAP_INDEX_CLASS)
|
||||
val bootstrapIndexClass = hoodieConfig.getStringOrDefault(INDEX_CLASS_NAME)
|
||||
|
||||
var schema: String = null
|
||||
if (df.schema.nonEmpty) {
|
||||
@@ -363,18 +362,18 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
|
||||
if (!tableExists) {
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)
|
||||
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
|
||||
val partitionColumns = HoodieWriterUtils.getPartitionColumns(parameters)
|
||||
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
|
||||
val keyGenProp = hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS)
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val keyGenProp = hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.valueOf(tableType))
|
||||
.setTableName(tableName)
|
||||
.setRecordKeyFields(recordKeyFields)
|
||||
.setArchiveLogFolder(archiveLogFolder)
|
||||
.setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS))
|
||||
.setPayloadClassName(hoodieConfig.getStringOrDefault(PAYLOAD_CLASS_NAME))
|
||||
.setPreCombineField(hoodieConfig.getStringOrDefault(PRECOMBINE_FIELD, null))
|
||||
.setBootstrapIndexClass(bootstrapIndexClass)
|
||||
.setBootstrapBasePath(bootstrapBasePath)
|
||||
@@ -405,8 +404,8 @@ object HoodieSparkSqlWriter {
|
||||
instantTime: String,
|
||||
partitionColumns: String): (Boolean, common.util.Option[String]) = {
|
||||
val sparkContext = sqlContext.sparkContext
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
|
||||
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(),
|
||||
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean
|
||||
val dropPartitionColumns =
|
||||
parameters.getOrElse(DataSourceWriteOptions.DROP_PARTITION_COLUMNS.key(), DataSourceWriteOptions.DROP_PARTITION_COLUMNS.defaultValue()).toBoolean
|
||||
// register classes & schemas
|
||||
@@ -423,7 +422,7 @@ object HoodieSparkSqlWriter {
|
||||
if (parameters(INSERT_DROP_DUPS.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.key, schema.toString)
|
||||
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA_STRING.key, schema.toString)
|
||||
val writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path.get, tblName, mapAsJavaMap(params))
|
||||
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
|
||||
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
|
||||
@@ -533,21 +532,21 @@ object HoodieSparkSqlWriter {
|
||||
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER)
|
||||
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS)
|
||||
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL)
|
||||
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX,
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.defaultValue).toBoolean
|
||||
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE,
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
|
||||
hiveSyncConfig.partitionFields =
|
||||
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
|
||||
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
|
||||
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_ENABLE_PROP)
|
||||
hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.METADATA_VALIDATE_PROP)
|
||||
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
|
||||
hiveSyncConfig.verifyMetadataFileListing = hoodieConfig.getBoolean(HoodieMetadataConfig.VALIDATE_ENABLE)
|
||||
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
|
||||
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP).toBoolean
|
||||
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
|
||||
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean
|
||||
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean
|
||||
hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt
|
||||
|
||||
hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
|
||||
hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE)
|
||||
@@ -561,7 +560,7 @@ object HoodieSparkSqlWriter {
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
|
||||
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
|
||||
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
|
||||
// for backward compatibility
|
||||
if (hiveSyncEnabled) {
|
||||
|
||||
@@ -17,13 +17,12 @@
|
||||
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.{METADATA_ENABLE_PROP, METADATA_VALIDATE_PROP}
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.{ENABLE, VALIDATE_ENABLE}
|
||||
import org.apache.hudi.common.config.{HoodieConfig, TypedProperties}
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConversions.mapAsJavaMap
|
||||
import scala.collection.JavaConverters.{mapAsScalaMapConverter, _}
|
||||
|
||||
@@ -46,18 +45,18 @@ object HoodieWriterUtils {
|
||||
Map(OPERATION.key -> OPERATION.defaultValue,
|
||||
TABLE_TYPE.key -> TABLE_TYPE.defaultValue,
|
||||
PRECOMBINE_FIELD.key -> PRECOMBINE_FIELD.defaultValue,
|
||||
PAYLOAD_CLASS.key -> PAYLOAD_CLASS.defaultValue,
|
||||
PAYLOAD_CLASS_NAME.key -> PAYLOAD_CLASS_NAME.defaultValue,
|
||||
RECORDKEY_FIELD.key -> RECORDKEY_FIELD.defaultValue,
|
||||
PARTITIONPATH_FIELD.key -> PARTITIONPATH_FIELD.defaultValue,
|
||||
KEYGENERATOR_CLASS.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
METADATA_ENABLE_PROP.key -> METADATA_ENABLE_PROP.defaultValue.toString,
|
||||
METADATA_VALIDATE_PROP.key -> METADATA_VALIDATE_PROP.defaultValue.toString,
|
||||
KEYGENERATOR_CLASS_NAME.key -> DEFAULT_KEYGENERATOR_CLASS_OPT_VAL,
|
||||
ENABLE.key -> ENABLE.defaultValue.toString,
|
||||
VALIDATE_ENABLE.key -> VALIDATE_ENABLE.defaultValue.toString,
|
||||
COMMIT_METADATA_KEYPREFIX.key -> COMMIT_METADATA_KEYPREFIX.defaultValue,
|
||||
INSERT_DROP_DUPS.key -> INSERT_DROP_DUPS.defaultValue,
|
||||
STREAMING_RETRY_CNT.key -> STREAMING_RETRY_CNT.defaultValue,
|
||||
STREAMING_RETRY_INTERVAL_MS.key -> STREAMING_RETRY_INTERVAL_MS.defaultValue,
|
||||
STREAMING_IGNORE_FAILED_BATCH.key -> STREAMING_IGNORE_FAILED_BATCH.defaultValue,
|
||||
META_SYNC_CLIENT_TOOL_CLASS.key -> META_SYNC_CLIENT_TOOL_CLASS.defaultValue,
|
||||
META_SYNC_CLIENT_TOOL_CLASS_NAME.key -> META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue,
|
||||
HIVE_SYNC_ENABLED.key -> HIVE_SYNC_ENABLED.defaultValue,
|
||||
META_SYNC_ENABLED.key -> META_SYNC_ENABLED.defaultValue,
|
||||
HIVE_DATABASE.key -> HIVE_DATABASE.defaultValue,
|
||||
|
||||
@@ -42,26 +42,26 @@ object HoodieOptionConfig {
|
||||
val SQL_KEY_TABLE_PRIMARY_KEY: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("primaryKey")
|
||||
.withHoodieKey(DataSourceWriteOptions.RECORDKEY_FIELD.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
|
||||
.withTableConfigKey(HoodieTableConfig.RECORDKEY_FIELDS.key)
|
||||
.build()
|
||||
|
||||
val SQL_KEY_TABLE_TYPE: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("type")
|
||||
.withHoodieKey(DataSourceWriteOptions.TABLE_TYPE.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP.key)
|
||||
.withTableConfigKey(HoodieTableConfig.TYPE.key)
|
||||
.defaultValue(SQL_VALUE_TABLE_TYPE_COW)
|
||||
.build()
|
||||
|
||||
val SQL_KEY_PRECOMBINE_FIELD: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("preCombineField")
|
||||
.withHoodieKey(DataSourceWriteOptions.PRECOMBINE_FIELD.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
|
||||
.withTableConfigKey(HoodieTableConfig.PRECOMBINE_FIELD.key)
|
||||
.build()
|
||||
|
||||
val SQL_PAYLOAD_CLASS: HoodieOption[String] = buildConf()
|
||||
.withSqlKey("payloadClass")
|
||||
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS.key)
|
||||
.withTableConfigKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP.key)
|
||||
.withHoodieKey(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key)
|
||||
.withTableConfigKey(HoodieTableConfig.PAYLOAD_CLASS_NAME.key)
|
||||
.defaultValue(classOf[DefaultHoodieRecordPayload].getName)
|
||||
.build()
|
||||
|
||||
|
||||
@@ -17,22 +17,19 @@
|
||||
|
||||
package org.apache.spark.sql.hudi.command
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import java.util.{Locale, Properties}
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.hudi.common.model.HoodieFileFormat
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
|
||||
import org.apache.hudi.common.util.ValidationUtils
|
||||
import org.apache.hudi.hadoop.HoodieParquetInputFormat
|
||||
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat
|
||||
import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.{SPARK_VERSION, SparkConf}
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.avro.SchemaConverters
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
@@ -41,13 +38,16 @@ import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat,
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.hive.HiveClientUtils
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog._
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
import org.apache.spark.sql.hudi.HoodieOptionConfig
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
import org.apache.spark.sql.hudi.command.CreateHoodieTableCommand.{initTableIfNeed, isEmptyPath}
|
||||
import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
|
||||
import org.apache.spark.{SPARK_VERSION, SparkConf}
|
||||
|
||||
import java.util.{Locale, Properties}
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.mutable
|
||||
|
||||
/**
|
||||
@@ -106,28 +106,28 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
|
||||
// (By default this config is enable for spark sql)
|
||||
upgrateConfig = if (isNotHiveStyledPartitionTable(allPartitionPaths, table)) {
|
||||
upgrateConfig + (DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false")
|
||||
} else {
|
||||
upgrateConfig
|
||||
}
|
||||
upgrateConfig = if (isUrlEncodeDisable(allPartitionPaths, table)) {
|
||||
upgrateConfig + (DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key -> "false")
|
||||
} else {
|
||||
upgrateConfig
|
||||
}
|
||||
upgrateConfig = if (isUrlEncodeDisable(allPartitionPaths, table)) {
|
||||
upgrateConfig + (DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key -> "false")
|
||||
} else {
|
||||
upgrateConfig
|
||||
}
|
||||
|
||||
// Use the origin keygen to generate record key to keep the rowkey consistent with the old table for spark sql.
|
||||
// See SqlKeyGenerator#getRecordKey for detail.
|
||||
upgrateConfig = if (originTableConfig.contains(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key)) {
|
||||
upgrateConfig + (SqlKeyGenerator.ORIGIN_KEYGEN_CLASS -> originTableConfig(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key))
|
||||
} else {
|
||||
upgrateConfig = if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
|
||||
upgrateConfig + (SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
|
||||
} else {
|
||||
upgrateConfig
|
||||
}
|
||||
val options = originTableConfig ++ upgrateConfig ++ table.storage.properties
|
||||
}
|
||||
val options = originTableConfig ++ upgrateConfig ++ table.storage.properties
|
||||
|
||||
val userSpecifiedSchema = table.schema
|
||||
if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) {
|
||||
val userSpecifiedSchema = table.schema
|
||||
if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) {
|
||||
(addMetaFields(tableSchema.get), options)
|
||||
} else if (userSpecifiedSchema.nonEmpty) {
|
||||
} else if (userSpecifiedSchema.nonEmpty) {
|
||||
(addMetaFields(userSpecifiedSchema), options)
|
||||
} else {
|
||||
throw new IllegalArgumentException(s"Missing schema for Create Table: $tableName")
|
||||
@@ -392,19 +392,19 @@ object CreateHoodieTableCommand extends Logging {
|
||||
val tableName = table.identifier.table
|
||||
logInfo(s"Init hoodie.properties for $tableName")
|
||||
val tableOptions = HoodieOptionConfig.mappingSqlOptionToTableConfig(table.storage.properties)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key)
|
||||
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key)
|
||||
// Save all the table config to the hoodie.properties.
|
||||
val parameters = originTableConfig ++ tableOptions
|
||||
val properties = new Properties()
|
||||
properties.putAll(parameters.asJava)
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(properties)
|
||||
.setTableName(tableName)
|
||||
.setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString())
|
||||
.setPartitionFields(table.partitionColumnNames.mkString(","))
|
||||
.initTable(conf, location)
|
||||
properties.putAll(parameters.asJava)
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.fromProperties(properties)
|
||||
.setTableName(tableName)
|
||||
.setTableCreateSchema(SchemaConverters.toAvroType(table.schema).toString())
|
||||
.setPartitionFields(table.partitionColumnNames.mkString(","))
|
||||
.initTable(conf, location)
|
||||
}
|
||||
|
||||
def checkTableConfigEqual(originTableConfig: Map[String, String],
|
||||
|
||||
@@ -17,14 +17,13 @@
|
||||
|
||||
package org.apache.spark.sql.hudi.command
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions.OPERATION
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{DeleteFromTable, SubqueryAlias}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.DeleteFromTable
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.hudi.HoodieOptionConfig
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
@@ -67,14 +66,14 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
|
||||
withSparkConf(sparkSession, targetTable.storage.properties) {
|
||||
Map(
|
||||
"path" -> path,
|
||||
KEYGENERATOR_CLASS.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
TABLE_NAME.key -> tableId.table,
|
||||
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
TBL_NAME.key -> tableId.table,
|
||||
OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING.key -> "true",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
@@ -17,15 +17,14 @@
|
||||
|
||||
package org.apache.spark.sql.hudi.command
|
||||
|
||||
import java.util.Properties
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.{GenericRecord, IndexedRecord}
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord}
|
||||
import org.apache.hudi.common.util.{Option => HOption}
|
||||
import org.apache.hudi.exception.HoodieDuplicateKeyException
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.exception.HoodieDuplicateKeyException
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.sql.InsertMode
|
||||
@@ -33,22 +32,24 @@ import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWrit
|
||||
import org.apache.spark.internal.Logging
|
||||
import org.apache.spark.sql.catalyst.catalog.CatalogTable
|
||||
import org.apache.spark.sql.catalyst.expressions.{Alias, Literal}
|
||||
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.execution.datasources.LogicalRelation
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, HoodieSqlUtils}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Dataset, Row, SaveMode, SparkSession}
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
/**
|
||||
* Command for insert into hoodie table.
|
||||
*/
|
||||
case class InsertIntoHoodieTableCommand(
|
||||
logicalRelation: LogicalRelation,
|
||||
query: LogicalPlan,
|
||||
partition: Map[String, Option[String]],
|
||||
overwrite: Boolean)
|
||||
logicalRelation: LogicalRelation,
|
||||
query: LogicalPlan,
|
||||
partition: Map[String, Option[String]],
|
||||
overwrite: Boolean)
|
||||
extends RunnableCommand {
|
||||
|
||||
override def run(sparkSession: SparkSession): Seq[Row] = {
|
||||
@@ -262,13 +263,13 @@ object InsertIntoHoodieTableCommand extends Logging {
|
||||
Map(
|
||||
"path" -> path,
|
||||
TABLE_TYPE.key -> tableType,
|
||||
TABLE_NAME.key -> table.identifier.table,
|
||||
TBL_NAME.key -> table.identifier.table,
|
||||
PRECOMBINE_FIELD.key -> tableSchema.fields.last.name,
|
||||
OPERATION.key -> operation,
|
||||
KEYGENERATOR_CLASS.key -> keyGenClass,
|
||||
KEYGENERATOR_CLASS_NAME.key -> keyGenClass,
|
||||
RECORDKEY_FIELD.key -> primaryColumns.mkString(","),
|
||||
PARTITIONPATH_FIELD.key -> partitionFields,
|
||||
PAYLOAD_CLASS.key -> payloadClassName,
|
||||
PAYLOAD_CLASS_NAME.key -> payloadClassName,
|
||||
ENABLE_ROW_WRITER.key -> enableBulkInsert.toString,
|
||||
HoodieWriteConfig.COMBINE_BEFORE_INSERT.key -> isPrimaryKeyTable.toString,
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
@@ -276,13 +277,13 @@ object InsertIntoHoodieTableCommand extends Logging {
|
||||
HIVE_USE_JDBC.key -> "false",
|
||||
HIVE_DATABASE.key -> table.identifier.database.getOrElse("default"),
|
||||
HIVE_TABLE.key -> table.identifier.table,
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING.key -> "true",
|
||||
HIVE_PARTITION_FIELDS.key -> partitionFields,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING.key -> "true",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
@@ -17,24 +17,25 @@
|
||||
|
||||
package org.apache.spark.sql.hudi.command
|
||||
|
||||
import java.util.Base64
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils, SparkAdapterSupport}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BoundReference, Cast, EqualTo, Expression, Literal}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, MergeIntoTable, SubqueryAlias, UpdateAction}
|
||||
import org.apache.spark.sql.catalyst.plans.logical._
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
import org.apache.spark.sql.types.{BooleanType, StructType}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, SerDeUtils}
|
||||
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload
|
||||
import org.apache.spark.sql.hudi.command.payload.ExpressionPayload._
|
||||
import org.apache.spark.sql.hudi.{HoodieOptionConfig, SerDeUtils}
|
||||
import org.apache.spark.sql.types.{BooleanType, StructType}
|
||||
|
||||
import java.util.Base64
|
||||
|
||||
/**
|
||||
* The Command for hoodie MergeIntoTable.
|
||||
@@ -431,24 +432,24 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
|
||||
Map(
|
||||
"path" -> path,
|
||||
RECORDKEY_FIELD.key -> targetKey2SourceExpression.keySet.mkString(","),
|
||||
KEYGENERATOR_CLASS.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
|
||||
TABLE_NAME.key -> targetTableName,
|
||||
TBL_NAME.key -> targetTableName,
|
||||
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
PAYLOAD_CLASS.key -> classOf[ExpressionPayload].getCanonicalName,
|
||||
PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName,
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_USE_JDBC.key -> "false",
|
||||
HIVE_DATABASE.key -> targetTableDb,
|
||||
HIVE_TABLE.key -> targetTableName,
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING.key -> "true",
|
||||
HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING.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",
|
||||
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
})
|
||||
|
||||
@@ -43,12 +43,12 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props)
|
||||
}
|
||||
// The origin key generator class for this table.
|
||||
private lazy val originKeyGen = {
|
||||
val beforeKeyGenClassName = props.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS, null)
|
||||
val beforeKeyGenClassName = props.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null)
|
||||
if (beforeKeyGenClassName != null) {
|
||||
val keyGenProps = new TypedProperties()
|
||||
keyGenProps.putAll(props)
|
||||
keyGenProps.remove(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS)
|
||||
keyGenProps.put(HoodieWriteConfig.KEYGENERATOR_CLASS.key, beforeKeyGenClassName)
|
||||
keyGenProps.remove(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
|
||||
keyGenProps.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, beforeKeyGenClassName)
|
||||
Some(KeyGenUtils.createKeyGeneratorByClassName(keyGenProps))
|
||||
} else {
|
||||
None
|
||||
@@ -118,7 +118,7 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props)
|
||||
|
||||
object SqlKeyGenerator {
|
||||
val PARTITION_SCHEMA = "hoodie.sql.partition.schema"
|
||||
val ORIGIN_KEYGEN_CLASS = "hoodie.sql.origin.keygen.class"
|
||||
val ORIGIN_KEYGEN_CLASS_NAME = "hoodie.sql.origin.keygen.class"
|
||||
private val timestampTimeFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
|
||||
private val sqlTimestampFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S")
|
||||
}
|
||||
|
||||
@@ -17,16 +17,16 @@
|
||||
|
||||
package org.apache.spark.sql.hudi.command
|
||||
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.model.HoodieRecord
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TABLE_NAME
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Assignment, SubqueryAlias, UpdateTable}
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Assignment, UpdateTable}
|
||||
import org.apache.spark.sql.execution.command.RunnableCommand
|
||||
import org.apache.spark.sql.hudi.HoodieOptionConfig
|
||||
import org.apache.spark.sql.hudi.HoodieSqlUtils._
|
||||
@@ -95,9 +95,9 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
|
||||
Map(
|
||||
"path" -> path,
|
||||
RECORDKEY_FIELD.key -> primaryColumns.mkString(","),
|
||||
KEYGENERATOR_CLASS.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
PRECOMBINE_FIELD.key -> primaryColumns.head, //set the default preCombine field.
|
||||
TABLE_NAME.key -> tableId.table,
|
||||
TBL_NAME.key -> tableId.table,
|
||||
OPERATION.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
@@ -108,9 +108,9 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
|
||||
HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","),
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
URL_ENCODE_PARTITIONING.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP.key -> "true",
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_STYLE_PARTITIONING.key -> "true",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM.key -> "200",
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
|
||||
)
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user