1
0

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:
Udit Mehrotra
2021-08-19 13:36:40 -07:00
committed by GitHub
parent 37c29e75dc
commit c350d05dd3
137 changed files with 3460 additions and 1527 deletions

View File

@@ -185,7 +185,7 @@ public class DataSourceUtils {
return builder.forTable(tblName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS().key()))
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_NAME().key()))
.withInlineCompaction(inlineCompact).build())
.withClusteringConfig(HoodieClusteringConfig.newBuilder()
.withInlineClustering(inlineClusteringEnabled)
@@ -300,10 +300,10 @@ public class DataSourceUtils {
DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE().defaultValue()));
hiveSyncConfig.ignoreExceptions = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().key(),
DataSourceWriteOptions.HIVE_IGNORE_EXCEPTIONS().defaultValue()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().key(),
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX().defaultValue()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().key(),
DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP().defaultValue()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().key(),
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE().defaultValue()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().key(),
DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP_TYPE().defaultValue()));
return hiveSyncConfig;
}
}

View File

@@ -106,7 +106,7 @@ public class BulkInsertDataInternalWriterHelper {
private Option<BuiltinKeyGenerator> getKeyGenerator(Properties properties) {
TypedProperties typedProperties = new TypedProperties();
typedProperties.putAll(properties);
if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
if (properties.get(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key()).equals(NonpartitionedKeyGenerator.class.getName())) {
return Option.empty(); // Do not instantiate NonPartitionKeyGen
} else {
try {

View File

@@ -204,7 +204,7 @@ object DataSourceWriteOptions {
val partitionColumns = optParams.get(SparkDataSourceUtils.PARTITIONING_COLUMNS_KEY)
.map(SparkDataSourceUtils.decodePartitioningColumns)
.getOrElse(Nil)
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS.key(),
val keyGeneratorClass = optParams.getOrElse(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key(),
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL)
val partitionPathField =
@@ -244,7 +244,7 @@ object DataSourceWriteOptions {
* Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting.
* This will render any value set for `PRECOMBINE_FIELD_OPT_VAL` in-effective
*/
val PAYLOAD_CLASS = HoodieWriteConfig.WRITE_PAYLOAD_CLASS
val PAYLOAD_CLASS_NAME = HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
@@ -267,7 +267,7 @@ object DataSourceWriteOptions {
*/
val HIVE_STYLE_PARTITIONING = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING
val KEYGENERATOR_CLASS = ConfigProperty.key("hoodie.datasource.write.keygenerator.class")
val KEYGENERATOR_CLASS_NAME = ConfigProperty.key("hoodie.datasource.write.keygenerator.class")
.defaultValue(classOf[SimpleKeyGenerator].getName)
.withDocumentation("Key generator class, that implements `org.apache.hudi.keygen.KeyGenerator`")
@@ -328,7 +328,7 @@ object DataSourceWriteOptions {
.withDocumentation("Config to indicate whether to ignore any non exception error (e.g. writestatus error)"
+ " within a streaming microbatch")
val META_SYNC_CLIENT_TOOL_CLASS: ConfigProperty[String] = ConfigProperty
val META_SYNC_CLIENT_TOOL_CLASS_NAME: ConfigProperty[String] = ConfigProperty
.key("hoodie.meta.sync.client.tool.class")
.defaultValue(classOf[HiveSyncTool].getName)
.withDocumentation("Sync tool class name used to sync to metastore. Defaults to Hive.")
@@ -403,7 +403,7 @@ object DataSourceWriteOptions {
.defaultValue("false")
.withDocumentation("")
/* @deprecated We should use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
/** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
@Deprecated
val HIVE_USE_JDBC: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.use_jdbc")
@@ -421,12 +421,12 @@ object DataSourceWriteOptions {
.defaultValue("false")
.withDocumentation("")
val HIVE_SKIP_RO_SUFFIX: ConfigProperty[String] = ConfigProperty
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.skip_ro_suffix")
.defaultValue("false")
.withDocumentation("Skip the _ro suffix for Read optimized table, when registering")
val HIVE_SUPPORT_TIMESTAMP: ConfigProperty[String] = ConfigProperty
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = ConfigProperty
.key("hoodie.datasource.hive_sync.support_timestamp")
.defaultValue("false")
.withDocumentation("INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type. " +
@@ -514,9 +514,12 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */
@Deprecated
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = STREAMING_IGNORE_FAILED_BATCH.defaultValue()
/** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS} and its methods instead */
/** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS_NAME} and its methods instead */
@Deprecated
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS.defaultValue()
val META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.key()
/** @deprecated Use {@link META_SYNC_CLIENT_TOOL_CLASS_NAME} and its methods instead */
@Deprecated
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue()
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
@Deprecated
val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key()
@@ -550,10 +553,10 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link KEYGENERATOR_CLASS} and its methods instead */
@Deprecated
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = KEYGENERATOR_CLASS.defaultValue()
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = KEYGENERATOR_CLASS_NAME.defaultValue()
/** @deprecated Use {@link KEYGENERATOR_CLASS} and its methods instead */
@Deprecated
val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS.key()
val KEYGENERATOR_CLASS_OPT_KEY = HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key()
/** @deprecated Use {@link ENABLE_ROW_WRITER} and its methods instead */
@Deprecated
val ENABLE_ROW_WRITER_OPT_KEY = ENABLE_ROW_WRITER.key()
@@ -622,12 +625,12 @@ object DataSourceWriteOptions {
@Deprecated
val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = PRECOMBINE_FIELD.defaultValue()
/** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS} and its methods instead */
/** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME} and its methods instead */
@Deprecated
val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS.key()
/** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS} and its methods instead */
val PAYLOAD_CLASS_OPT_KEY = HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME.key()
/** @deprecated Use {@link HoodieWriteConfig.WRITE_PAYLOAD_CLASS_NAME} and its methods instead */
@Deprecated
val DEFAULT_PAYLOAD_OPT_VAL = PAYLOAD_CLASS.defaultValue()
val DEFAULT_PAYLOAD_OPT_VAL = PAYLOAD_CLASS_NAME.defaultValue()
/** @deprecated Use {@link TABLE_TYPE} and its methods instead */
@Deprecated
@@ -638,7 +641,7 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link TABLE_TYPE} and its methods instead */
@Deprecated
val STORAGE_TYPE_OPT = "hoodie.datasource.write.storage.type"
val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
@Deprecated
val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
@Deprecated
@@ -698,12 +701,18 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
@Deprecated
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.defaultValue()
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX} and its methods instead */
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX.defaultValue()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP} and its methods instead */
val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP.defaultValue()
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
@Deprecated
val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key()
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
@Deprecated
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
/** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */
@Deprecated
val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key()
@@ -713,6 +722,8 @@ object DataSourceWriteOptions {
/** @deprecated Use {@link KAFKA_AVRO_VALUE_DESERIALIZER_CLASS} and its methods instead */
@Deprecated
val KAFKA_AVRO_VALUE_DESERIALIZER = KAFKA_AVRO_VALUE_DESERIALIZER_CLASS.key()
@Deprecated
val SCHEMA_PROVIDER_CLASS_PROP = "hoodie.deltastreamer.schemaprovider.class"
}
object DataSourceOptionsHelper {
@@ -723,13 +734,13 @@ object DataSourceOptionsHelper {
val allConfigsWithAlternatives = List(
DataSourceReadOptions.QUERY_TYPE,
DataSourceWriteOptions.TABLE_TYPE,
HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP,
HoodieTableConfig.HOODIE_LOG_FILE_FORMAT_PROP
HoodieTableConfig.BASE_FILE_FORMAT,
HoodieTableConfig.LOG_FILE_FORMAT
)
// put all the deprecated configs here
val allDeprecatedConfigs: Set[String] = Set(
ConsistencyGuardConfig.CONSISTENCY_CHECK_ENABLED_PROP.key
ConsistencyGuardConfig.ENABLE.key
)
// maps the deprecated config name to its latest name

View File

@@ -72,10 +72,10 @@ public class HoodieBulkInsertInternalWriterTestBase extends HoodieClientTestHarn
protected HoodieWriteConfig getWriteConfig(boolean populateMetaFields) {
Properties properties = new Properties();
if (!populateMetaFields) {
properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), SimpleKeyGenerator.class.getName());
properties.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), SimpleKeyGenerator.class.getName());
properties.setProperty(DataSourceWriteOptions.RECORDKEY_FIELD().key(), SparkDatasetTestUtils.RECORD_KEY_FIELD_NAME);
properties.setProperty(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), SparkDatasetTestUtils.PARTITION_PATH_FIELD_NAME);
properties.setProperty(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), "false");
properties.setProperty(HoodieTableConfig.POPULATE_META_FIELDS.key(), "false");
}
return getConfigBuilder(basePath).withProperties(properties).build();
}

View File

@@ -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();

View File

@@ -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
}

View File

@@ -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) {

View File

@@ -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,

View File

@@ -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()

View File

@@ -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],

View File

@@ -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
)
}

View File

@@ -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
)
}

View File

@@ -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
)
})

View File

@@ -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")
}

View File

@@ -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
)
}

View File

@@ -16,7 +16,6 @@
* limitations under the License.
*/
import org.apache.hadoop.fs.Path;
import org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieDataSourceHelpers;
@@ -33,6 +32,7 @@ import org.apache.hudi.keygen.SimpleKeyGenerator;
import com.beust.jcommander.JCommander;
import com.beust.jcommander.Parameter;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaSparkContext;
@@ -150,9 +150,9 @@ public class HoodieJavaApp {
// use to combine duplicate records in input/with disk val
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
// Used by hive sync and queries
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName)
// Add Key Extractor
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
@@ -179,13 +179,13 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);
@@ -207,13 +207,13 @@ public class HoodieJavaApp {
.option(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "_row_key")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName()) // Add Key Extractor
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "false")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).mode(SaveMode.Append);
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).mode(SaveMode.Append);
updateHiveSyncConfig(writer);
writer.save(tablePath);

View File

@@ -175,9 +175,9 @@ public class HoodieJavaGenerateApp {
// use to combine duplicate records in input/with disk val
.option(DataSourceWriteOptions.PRECOMBINE_FIELD().key(), "timestamp")
// Used by hive sync and queries
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName)
// Add Key Extractor
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(),
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(),
nonPartitionedTable ? NonpartitionedKeyGenerator.class.getCanonicalName()
: SimpleKeyGenerator.class.getCanonicalName())
.mode(commitType);

View File

@@ -16,7 +16,6 @@
* limitations under the License.
*/
import java.util.stream.Collectors;
import org.apache.hudi.DataSourceReadOptions;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.HoodieDataSourceHelpers;
@@ -43,13 +42,14 @@ import org.apache.spark.sql.SaveMode;
import org.apache.spark.sql.SparkSession;
import org.apache.spark.sql.streaming.DataStreamWriter;
import org.apache.spark.sql.streaming.OutputMode;
import org.apache.spark.sql.streaming.StreamingQuery;
import org.apache.spark.sql.streaming.Trigger;
import java.util.List;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.Future;
import org.apache.spark.sql.streaming.StreamingQuery;
import java.util.stream.Collectors;
import static org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings;
@@ -363,7 +363,7 @@ public class HoodieJavaStreamingApp {
.option(HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS.key(), "1")
.option(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE().key(), "true")
.option(DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE().key(), "true")
.option(HoodieWriteConfig.TABLE_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.option(HoodieWriteConfig.TBL_NAME.key(), tableName).option("checkpointLocation", checkpointLocation)
.outputMode(OutputMode.Append());
updateHiveSyncConfig(writer);

View File

@@ -18,12 +18,6 @@
package org.apache.hudi.functional;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.JobConf;
import org.apache.hudi.DataSourceWriteOptions;
import org.apache.hudi.avro.model.HoodieFileStatus;
import org.apache.hudi.client.SparkRDDWriteClient;
@@ -49,6 +43,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.testutils.RawTripTestPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ParquetReaderIterator;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieBootstrapConfig;
import org.apache.hudi.config.HoodieCompactionConfig;
@@ -58,11 +53,17 @@ import org.apache.hudi.hadoop.HoodieParquetInputFormat;
import org.apache.hudi.hadoop.realtime.HoodieParquetRealtimeInputFormat;
import org.apache.hudi.index.HoodieIndex.IndexType;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.common.util.PartitionPathEncodeUtils;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.table.action.bootstrap.BootstrapUtils;
import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils;
import org.apache.avro.Schema;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.io.LongWritable;
import org.apache.hadoop.mapred.JobConf;
import org.apache.parquet.avro.AvroParquetReader;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroSchemaConverter;
@@ -374,7 +375,7 @@ public class TestBootstrap extends HoodieClientTestBase {
List<GenericRecord> records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.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());
@@ -393,7 +394,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.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());
@@ -410,7 +411,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES,
true, HoodieRecord.HOODIE_META_COLUMNS);
@@ -428,7 +429,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
HoodieRecord.HOODIE_META_COLUMNS);
@@ -444,7 +445,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, roJobConf, false, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));
@@ -462,7 +463,7 @@ public class TestBootstrap extends HoodieClientTestBase {
records = HoodieMergeOnReadTestUtils.getRecordsUsingInputFormat(
jsc.hadoopConfiguration(),
FSUtils.getAllPartitionPaths(context, basePath, HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS,
HoodieMetadataConfig.METADATA_VALIDATE_PROP.defaultValue(), false).stream()
HoodieMetadataConfig.VALIDATE_ENABLE.defaultValue(), false).stream()
.map(f -> basePath + "/" + f).collect(Collectors.toList()),
basePath, rtJobConf, true, schema, TRIP_HIVE_COLUMN_TYPES, true,
Arrays.asList("_row_key"));

View File

@@ -214,12 +214,12 @@ 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().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");
props.put(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "partition");
} else {
if (setKeyGen) {
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
props.put(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), "org.apache.hudi.keygen.SimpleKeyGenerator");
}
if (setRecordKey) {
props.put(DataSourceWriteOptions.RECORDKEY_FIELD().key(), "_row_key");

View File

@@ -22,9 +22,8 @@ import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.client.SparkRDDWriteClient
import org.apache.hudi.common.config.HoodieConfig
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord, HoodieRecordPayload, HoodieTableType, WriteOperationType}
import org.apache.hudi.common.table.HoodieTableConfig
import org.apache.hudi.common.model._
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException
@@ -73,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.key -> "hoodie_test_tbl")
val options = Map("path" -> "hoodie/test/path", HoodieWriteConfig.TBL_NAME.key -> "hoodie_test_tbl")
val e = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.ErrorIfExists, options,
session.emptyDataFrame))
assert(e.getMessage.contains("spark.serializer"))
@@ -89,7 +88,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
val hoodieFooTableName = "hoodie_foo_tbl"
//create a new table
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
@@ -98,7 +97,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.key -> "hoodie_bar_tbl",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl",
"hoodie.insert.shuffle.parallelism" -> "4",
"hoodie.upsert.shuffle.parallelism" -> "4")
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
@@ -151,8 +150,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
.updated(HoodieWriteConfig.BULKINSERT_SORT_MODE.key(), sortMode.name())
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name())
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
@@ -200,7 +199,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieWriteConfig.BULKINSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
@@ -352,15 +351,15 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
try {
val hoodieFooTableName = "hoodie_foo_tbl"
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
HoodieWriteConfig.BASE_FILE_FORMAT.key -> baseFileFormat,
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
HoodieWriteConfig.INSERT_PARALLELISM.key -> "4",
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "4",
DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL,
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> classOf[SimpleKeyGenerator].getCanonicalName)
HoodieTableConfig.POPULATE_META_FIELDS.key() -> String.valueOf(populateMetaFields),
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> classOf[SimpleKeyGenerator].getCanonicalName)
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts
@@ -428,14 +427,14 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
.save(srcPath.toAbsolutePath.toString)
val fooTableModifier = Map("path" -> path.toAbsolutePath.toString,
HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key -> srcPath.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
HoodieBootstrapConfig.BASE_PATH.key -> srcPath.toAbsolutePath.toString,
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key -> "4",
HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4",
DataSourceWriteOptions.OPERATION.key -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL,
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key -> classOf[NonpartitionedKeyGenerator].getCanonicalName)
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val client = spy(DataSourceUtils.createHoodieClient(
@@ -552,7 +551,7 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
"path" -> basePath,
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition",
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX.key -> "true",
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key -> "true",
DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> "true"
)
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
@@ -621,8 +620,8 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "col3",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "keyid",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "",
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test")
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.NonpartitionedKeyGenerator",
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test")
try {
val df = spark.range(0, 1000).toDF("keyid")
.withColumn("col3", expr("keyid"))
@@ -652,10 +651,10 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
df.write.mode(SaveMode.Overwrite).save(baseBootStrapPath)
spark.emptyDataFrame.write.format("hudi")
.options(options)
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key, baseBootStrapPath)
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
.option(HoodieBootstrapConfig.BASE_PATH.key, baseBootStrapPath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getCanonicalName)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM.key, "4")
.option(HoodieBootstrapConfig.PARALLELISM_VALUE.key, "4")
.mode(SaveMode.Overwrite).save(basePath)
df.write.format("hudi")
@@ -750,13 +749,13 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
def getCommonParams(path: java.nio.file.Path, hoodieFooTableName: String, tableType: String) : Map[String, String] = {
Map("path" -> path.toAbsolutePath.toString,
HoodieWriteConfig.TABLE_NAME.key -> hoodieFooTableName,
HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "1",
"hoodie.upsert.shuffle.parallelism" -> "1",
DataSourceWriteOptions.TABLE_TYPE.key -> tableType,
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.KEYGENERATOR_CLASS.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.SimpleKeyGenerator")
}
test("test Non partition table with metatable support") {
@@ -775,11 +774,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "col3")
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "keyid")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.option(DataSourceWriteOptions.OPERATION.key, "insert")
.option("hoodie.insert.shuffle.parallelism", "1")
.option("hoodie.metadata.enable", "true")
.option(HoodieWriteConfig.TABLE_NAME.key, "hoodie_test")
.option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
.mode(SaveMode.Overwrite).save(basePath)
// upsert same record again
val df_update = spark.range(0, 10).toDF("keyid")
@@ -790,11 +789,11 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers {
.option(DataSourceWriteOptions.PRECOMBINE_FIELD.key, "col3")
.option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "keyid")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.option(DataSourceWriteOptions.OPERATION.key, "upsert")
.option("hoodie.upsert.shuffle.parallelism", "1")
.option("hoodie.metadata.enable", "true")
.option(HoodieWriteConfig.TABLE_NAME.key, "hoodie_test")
.option(HoodieWriteConfig.TBL_NAME.key, "hoodie_test")
.mode(SaveMode.Append).save(basePath)
assert(spark.read.format("hudi").load(basePath).count() == 10)
assert(spark.read.format("hudi").load(basePath).where("age >= 2000").count() == 10)

View File

@@ -29,10 +29,10 @@ import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator.{Config, TimestampType}
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal}
import org.apache.spark.sql.execution.datasources.PartitionDirectory
import org.apache.spark.sql.types.StringType
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.BeforeEach
import org.junit.jupiter.params.ParameterizedTest
@@ -50,7 +50,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
var queryOpts = Map(
@@ -95,7 +95,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
inputDF1.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.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")
@@ -116,7 +116,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
inputDF1.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "partition:simple")
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -176,14 +176,14 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "dt,hh")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "false")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
.option(HoodieMetadataConfig.ENABLE.key, useMetaFileList)
.mode(SaveMode.Overwrite)
.save(basePath)
metaClient = HoodieTableMetaClient.reload(metaClient)
val fileIndex = HoodieFileIndex(spark, metaClient, None,
queryOpts ++ Map(HoodieMetadataConfig.METADATA_ENABLE_PROP.key -> useMetaFileList.toString))
queryOpts ++ Map(HoodieMetadataConfig.ENABLE.key -> useMetaFileList.toString))
val partitionFilter1 = And(
EqualTo(attribute("dt"), literal("2021-03-01")),
@@ -198,7 +198,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
assertEquals(getFileCountInPartitionPath("2021-03-01/10"), filesAfterPrune.size)
val readDF1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
.option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList)
.load(basePath)
assertEquals(10, readDF1.count())
assertEquals(5, readDF1.filter("dt = '2021-03-01' and hh = '10'").count())
@@ -213,9 +213,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "dt,hh")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "false")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key(), useMetaFileList)
.option(HoodieMetadataConfig.ENABLE.key(), useMetaFileList)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -235,7 +235,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
assertEquals(getFileCountInPartitionPaths("2021/03/01/10", "2021/03/02/10"),
filesAfterPrune2.length)
val readDF2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, useMetaFileList)
.option(HoodieMetadataConfig.ENABLE.key, useMetaFileList)
.load(basePath)
assertEquals(10, readDF2.count())

View File

@@ -56,7 +56,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
val verificationCol: String = "driver"
@@ -117,7 +117,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
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.key))
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(commonOpts(HoodieWriteConfig.TBL_NAME.key))
spark.sparkContext.getConf.registerKryoClasses(
Array(classOf[org.apache.avro.generic.GenericData],
classOf[org.apache.avro.Schema]))
@@ -166,7 +166,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -175,7 +175,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Snapshot query
val snapshotDF1 = spark.read.format("org.apache.hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF1.count())
@@ -185,13 +185,13 @@ class TestCOWDataSource extends HoodieClientTestBase {
updateDf.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
val commitInstantTime2 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val snapshotDF2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF2.count())
assertEquals(updatedVerificationVal, snapshotDF2.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
@@ -203,7 +203,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
@@ -212,7 +212,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// Snapshot Query
val snapshotDF3 = spark.read.format("org.apache.hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, snapshotDF3.count()) // still 100, since we only updated
@@ -242,7 +242,7 @@ 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.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
@@ -510,7 +510,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT.key, "true")
.option(HoodieWriteConfig.AUTO_COMMIT_ENABLE.key, "true")
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -523,7 +523,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
inputDF.write.format("hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, keyGenerator)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, keyGenerator)
.mode(SaveMode.Overwrite)
}
@@ -660,7 +660,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
@@ -668,7 +668,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
// query the partition by filter
val count1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath)
.filter("partition = '2016/03/15'")
.count()
@@ -677,7 +677,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
// query the partition by path
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
val count2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + s"/$partitionPath")
.count()
assertEquals(countIn20160315, count2)
@@ -689,7 +689,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
// Incremental query without "*" in path

View File

@@ -17,39 +17,39 @@
package org.apache.hudi.functional
import collection.JavaConverters._
import org.apache.hadoop.fs.FileSystem
import org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider
import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.table.timeline.HoodieTimeline
import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieCompactionConfig, HoodieWriteConfig}
import org.apache.hudi.keygen.SimpleKeyGenerator
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.sql.functions.{col, lit}
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.api.io.TempDir
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import java.time.Instant
import java.util.Collections
import scala.collection.JavaConverters._
class TestDataSourceForBootstrap {
var spark: SparkSession = _
val commonOpts: Map[String, String] = Map(
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",
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "4",
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "4",
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "4",
HoodieWriteConfig.BULKINSERT_PARALLELISM_VALUE.key -> "4",
HoodieWriteConfig.FINALIZE_WRITE_PARALLELISM_VALUE.key -> "4",
HoodieBootstrapConfig.PARALLELISM_VALUE.key -> "4",
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
var basePath: String = _
var srcPath: String = _
@@ -118,7 +118,7 @@ class TestDataSourceForBootstrap {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.option(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key, "org.apache.hudi.keygen.NonpartitionedKeyGenerator")
.mode(SaveMode.Append)
.save(basePath)
@@ -429,10 +429,10 @@ class TestDataSourceForBootstrap {
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "datestr")
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.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)
.option(HoodieBootstrapConfig.BASE_PATH.key, srcPath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
.option(HoodieBootstrapConfig.MODE_SELECTOR_CLASS_NAME.key, classOf[FullRecordBootstrapModeSelector].getName)
.option(HoodieBootstrapConfig.FULL_BOOTSTRAP_INPUT_PROVIDER_CLASS_NAME.key, classOf[SparkParquetBootstrapDataProvider].getName)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -480,8 +480,8 @@ class TestDataSourceForBootstrap {
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, tableType)
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, partitionColumns.getOrElse(""))
.option(HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH.key, srcPath)
.option(HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS.key, classOf[SimpleKeyGenerator].getName)
.option(HoodieBootstrapConfig.BASE_PATH.key, srcPath)
.option(HoodieBootstrapConfig.KEYGEN_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
.mode(SaveMode.Overwrite)
.save(basePath)

View File

@@ -17,12 +17,12 @@
package org.apache.hudi.functional
import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.spark.sql.{SaveMode, SparkSession}
import org.junit.jupiter.api.{AfterEach, BeforeEach}
import org.junit.jupiter.api.Assertions.assertEquals
import org.junit.jupiter.api.{AfterEach, BeforeEach}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.ValueSource
@@ -34,7 +34,7 @@ class TestEmptyCommit extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach override def setUp() {

View File

@@ -18,30 +18,28 @@
package org.apache.hudi.functional
import org.apache.hadoop.fs.Path
import scala.collection.JavaConverters._
import org.apache.hudi.DataSourceWriteOptions.{KEYGENERATOR_CLASS, PARTITIONPATH_FIELD, PAYLOAD_CLASS, PRECOMBINE_FIELD, RECORDKEY_FIELD}
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils}
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig}
import org.apache.hudi.index.HoodieIndex.IndexType
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
import org.apache.log4j.LogManager
import org.apache.spark.sql._
import org.apache.spark.sql.functions._
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
import org.junit.jupiter.params.ParameterizedTest
import org.junit.jupiter.params.provider.{CsvSource, ValueSource}
import scala.collection.JavaConversions._
import scala.collection.JavaConverters._
/**
* Tests on Spark DataSource for MOR table.
@@ -56,7 +54,7 @@ class TestMORDataSource extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
val verificationCol: String = "driver"
@@ -89,7 +87,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -98,7 +96,7 @@ class TestMORDataSource extends HoodieClientTestBase {
// Read RO View
val hudiRODF1 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, hudiRODF1.count()) // still 100, since we only updated
@@ -111,7 +109,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val inputDF2: Dataset[Row] = spark.read.json(spark.sparkContext.parallelize(records2, 2))
inputDF2.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
@@ -119,7 +117,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val updateCommitTime = HoodieDataSourceHelpers.latestCommit(fs, basePath)
val hudiSnapshotDF2 = spark.read.format("org.apache.hudi")
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
val updateCommitTimes = hudiSnapshotDF2.select("_hoodie_commit_time").distinct().collectAsList().map(r => r.getString(0)).toList
@@ -131,12 +129,12 @@ class TestMORDataSource extends HoodieClientTestBase {
inputDF3.write.format("org.apache.hudi")
.options(commonOpts)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
val hudiSnapshotDF3 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + "/*/*/*")
assertEquals(100, hudiSnapshotDF3.count())
assertEquals(updatedVerificationVal, hudiSnapshotDF3.filter(col("_row_key") === verificationRowKey).select(verificationCol).first.getString(0))
@@ -420,7 +418,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option("hoodie.compact.inline", "false") // else fails due to compaction & deltacommit instant times being same
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.PAYLOAD_CLASS.key, classOf[DefaultHoodieRecordPayload].getName)
.option(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getName)
.mode(SaveMode.Overwrite)
.save(basePath)
val hudiSnapshotDF1 = spark.read.format("org.apache.hudi")
@@ -582,12 +580,12 @@ class TestMORDataSource extends HoodieClientTestBase {
df.write.format("org.apache.hudi")
.options(commonOpts)
// use DefaultHoodieRecordPayload here
.option(PAYLOAD_CLASS.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
.option(PAYLOAD_CLASS_NAME.key, classOf[DefaultHoodieRecordPayload].getCanonicalName)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
.mode(SaveMode.Append)
.save(basePath)
}
@@ -639,7 +637,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Overwrite)
.save(basePath)
val commitInstantTime1 = HoodieDataSourceHelpers.latestCommit(fs, basePath)
@@ -647,7 +645,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val countIn20160315 = records1.asScala.count(record => record.getPartitionPath == "2016/03/15")
// query the partition by filter
val count1 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath)
.filter("partition = '2016/03/15'")
.count()
@@ -656,7 +654,7 @@ class TestMORDataSource extends HoodieClientTestBase {
// query the partition by path
val partitionPath = if (partitionEncode) "2016%2F03%2F15" else "2016/03/15"
val count2 = spark.read.format("hudi")
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.load(basePath + s"/$partitionPath")
.count()
assertEquals(countIn20160315, count2)
@@ -669,7 +667,7 @@ class TestMORDataSource extends HoodieClientTestBase {
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, partitionEncode)
.option(HoodieMetadataConfig.METADATA_ENABLE_PROP.key, isMetadataEnabled)
.option(HoodieMetadataConfig.ENABLE.key, isMetadataEnabled)
.mode(SaveMode.Append)
.save(basePath)
// Incremental query without "*" in path

View File

@@ -21,7 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions
import org.apache.hudi.DataSourceWriteOptions.{PRECOMBINE_FIELD, RECORDKEY_FIELD}
import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM, INSERT_PARALLELISM, TABLE_NAME, UPSERT_PARALLELISM}
import org.apache.hudi.config.HoodieWriteConfig.{DELETE_PARALLELISM_VALUE, INSERT_PARALLELISM_VALUE, TBL_NAME, UPSERT_PARALLELISM_VALUE}
import org.apache.log4j.Level
import org.apache.spark.sql.streaming.StreamTest
import org.apache.spark.sql.{Row, SaveMode}
@@ -32,9 +32,9 @@ class TestStreamingSource extends StreamTest {
private val commonOptions = Map(
RECORDKEY_FIELD.key -> "id",
PRECOMBINE_FIELD.key -> "ts",
INSERT_PARALLELISM.key -> "4",
UPSERT_PARALLELISM.key -> "4",
DELETE_PARALLELISM.key -> "4"
INSERT_PARALLELISM_VALUE.key -> "4",
UPSERT_PARALLELISM_VALUE.key -> "4",
DELETE_PARALLELISM_VALUE.key -> "4"
)
private val columns = Seq("id", "name", "price", "ts")
@@ -50,7 +50,7 @@ class TestStreamingSource extends StreamTest {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(COPY_ON_WRITE)
.setTableName(getTableName(tablePath))
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS.defaultValue)
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
.initTable(spark.sessionState.newHadoopConf(), tablePath)
addData(tablePath, Seq(("1", "a1", "10", "000")))
@@ -100,7 +100,7 @@ class TestStreamingSource extends StreamTest {
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(MERGE_ON_READ)
.setTableName(getTableName(tablePath))
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS.defaultValue)
.setPayloadClassName(DataSourceWriteOptions.PAYLOAD_CLASS_NAME.defaultValue)
.initTable(spark.sessionState.newHadoopConf(), tablePath)
addData(tablePath, Seq(("1", "a1", "10", "000")))
@@ -143,7 +143,7 @@ class TestStreamingSource extends StreamTest {
.write
.format("org.apache.hudi")
.options(commonOptions)
.option(TABLE_NAME.key, getTableName(inputPath))
.option(TBL_NAME.key, getTableName(inputPath))
.mode(SaveMode.Append)
.save(inputPath)
}

View File

@@ -50,7 +50,7 @@ class TestStructuredStreaming extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach override def setUp() {
@@ -194,11 +194,11 @@ class TestStructuredStreaming extends HoodieClientTestBase {
def getClusteringOpts(isInlineClustering: String, isAsyncClustering: String, isAsyncCompaction: String,
clusteringNumCommit: String, fileMaxRecordNum: Int):Map[String, String] = {
commonOpts + (HoodieClusteringConfig.INLINE_CLUSTERING.key -> isInlineClustering,
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMIT.key -> clusteringNumCommit,
HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key -> clusteringNumCommit,
DataSourceWriteOptions.ASYNC_CLUSTERING_ENABLE.key -> isAsyncClustering,
DataSourceWriteOptions.ASYNC_COMPACT_ENABLE.key -> isAsyncCompaction,
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMIT_PROP.key -> clusteringNumCommit,
HoodieStorageConfig.PARQUET_FILE_MAX_BYTES.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
HoodieClusteringConfig.ASYNC_CLUSTERING_MAX_COMMITS.key -> clusteringNumCommit,
HoodieStorageConfig.PARQUET_MAX_FILE_SIZE.key -> dataGen.getEstimatedFileSizeInBytes(fileMaxRecordNum).toString
)
}

View File

@@ -17,13 +17,13 @@
package org.apache.hudi.functional
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator}
import org.apache.hudi.testutils.HoodieClientTestBase
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions}
import org.apache.spark.sql.{Row, SaveMode, SparkSession}
import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue}
import org.junit.jupiter.api.{AfterEach, BeforeEach}
@@ -42,7 +42,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "_row_key",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "partition",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "timestamp",
HoodieWriteConfig.TABLE_NAME.key -> "hoodie_test"
HoodieWriteConfig.TBL_NAME.key -> "hoodie_test"
)
@BeforeEach override def setUp() {
@@ -74,7 +74,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -88,7 +88,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
.mode(SaveMode.Append)
.save(basePath)
metaClient.reloadActiveTimeline()
@@ -102,7 +102,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
.mode(SaveMode.Append)
.save(basePath)
metaClient.reloadActiveTimeline()
@@ -148,7 +148,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "dt")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.mode(SaveMode.Overwrite)
.save(basePath)
@@ -162,7 +162,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "dt")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.mode(SaveMode.Append)
.save(basePath)
metaClient.reloadActiveTimeline()
@@ -176,7 +176,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "version")
.option(PARTITIONPATH_FIELD.key, "dt")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.mode(SaveMode.Append)
.save(basePath)
metaClient.reloadActiveTimeline()

View File

@@ -18,8 +18,6 @@
package org.apache.spark.sql.hudi
import org.apache.hudi.DataSourceWriteOptions._
import scala.collection.JavaConverters._
import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.config.HoodieWriteConfig
@@ -28,7 +26,9 @@ import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator,
import org.apache.spark.sql.SaveMode
import org.apache.spark.sql.catalyst.TableIdentifier
import org.apache.spark.sql.catalyst.catalog.CatalogTableType
import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField}
import org.apache.spark.sql.types._
import scala.collection.JavaConverters._
class TestCreateTable extends TestHoodieSqlBase {
@@ -288,14 +288,14 @@ class TestCreateTable extends TestHoodieSqlBase {
val df = Seq((1, "a1", 10, 1000, partitionValue)).toDF("id", "name", "value", "ts", "dt")
// Write a table by spark dataframe.
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "dt")
.option(KEYGENERATOR_CLASS.key, classOf[SimpleKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.option(KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
.mode(SaveMode.Overwrite)
.save(tablePath)
@@ -319,9 +319,9 @@ class TestCreateTable extends TestHoodieSqlBase {
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("dt")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
assertResult("dt")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$partitionValue')")
@@ -365,14 +365,14 @@ class TestCreateTable extends TestHoodieSqlBase {
val df = Seq((1, "a1", 10, 1000, day, 12)).toDF("id", "name", "value", "ts", "day", "hh")
// Write a table by spark dataframe.
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
.option(TABLE_TYPE.key, MOR_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "day,hh")
.option(KEYGENERATOR_CLASS.key, classOf[ComplexKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.option(KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
.mode(SaveMode.Overwrite)
.save(tablePath)
@@ -396,9 +396,9 @@ class TestCreateTable extends TestHoodieSqlBase {
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("day,hh")(properties(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
assertResult("day,hh")(properties(HoodieTableConfig.PARTITION_FIELDS.key))
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000, '$day', 12)")
@@ -440,14 +440,14 @@ class TestCreateTable extends TestHoodieSqlBase {
import spark.implicits._
val df = Seq((1, "a1", 10, 1000)).toDF("id", "name", "value", "ts")
df.write.format("hudi")
.option(HoodieWriteConfig.TABLE_NAME.key, tableName)
.option(HoodieWriteConfig.TBL_NAME.key, tableName)
.option(TABLE_TYPE.key, COW_TABLE_TYPE_OPT_VAL)
.option(RECORDKEY_FIELD.key, "id")
.option(PRECOMBINE_FIELD.key, "ts")
.option(PARTITIONPATH_FIELD.key, "")
.option(KEYGENERATOR_CLASS.key, classOf[NonpartitionedKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM.key, "1")
.option(KEYGENERATOR_CLASS_NAME.key, classOf[NonpartitionedKeyGenerator].getName)
.option(HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key, "1")
.option(HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key, "1")
.mode(SaveMode.Overwrite)
.save(tmp.getCanonicalPath)
@@ -470,8 +470,8 @@ class TestCreateTable extends TestHoodieSqlBase {
.setConf(spark.sessionState.newHadoopConf())
.build()
val properties = metaClient.getTableConfig.getProps.asScala.toMap
assertResult(true)(properties.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA.key))
assertResult("ts")(properties(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP.key))
assertResult(true)(properties.contains(HoodieTableConfig.CREATE_SCHEMA.key))
assertResult("ts")(properties(HoodieTableConfig.PRECOMBINE_FIELD.key))
// Test insert into
spark.sql(s"insert into $tableName values(2, 'a2', 10, 1000)")

View File

@@ -61,15 +61,15 @@ public class DefaultSource extends BaseDefaultSource implements DataSourceV2,
DataSourceOptions options) {
String instantTime = options.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY).get();
String path = options.get("path").get();
String tblName = options.get(HoodieWriteConfig.TABLE_NAME.key()).get();
boolean populateMetaFields = options.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()));
String tblName = options.get(HoodieWriteConfig.TBL_NAME.key()).get();
boolean populateMetaFields = options.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS.key(),
Boolean.parseBoolean(HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA.key()).get(), path, tblName, options.asMap());
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(options.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()).get(), path, tblName, options.asMap());
boolean arePartitionRecordsSorted = HoodieInternalConfig.getBulkInsertIsPartitionRecordsSorted(
options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).isPresent()
? options.get(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED).get() : null);
return Optional.of(new HoodieDataSourceInternalWriter(instantTime, config, schema, getSparkSession(),
getConfiguration(), options, populateMetaFields, arePartitionRecordsSorted));
getConfiguration(), options, populateMetaFields, arePartitionRecordsSorted));
}
}

View File

@@ -48,13 +48,13 @@ public class DefaultSource extends BaseDefaultSource implements TableProvider {
public Table getTable(StructType schema, Transform[] partitioning, Map<String, String> properties) {
String instantTime = properties.get(DataSourceInternalWriterHelper.INSTANT_TIME_OPT_KEY);
String path = properties.get("path");
String tblName = properties.get(HoodieWriteConfig.TABLE_NAME.key());
boolean populateMetaFields = Boolean.parseBoolean(properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(),
HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()));
String tblName = properties.get(HoodieWriteConfig.TBL_NAME.key());
boolean populateMetaFields = Boolean.parseBoolean(properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(),
HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()));
boolean arePartitionRecordsSorted = Boolean.parseBoolean(properties.getOrDefault(HoodieInternalConfig.BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED,
Boolean.toString(HoodieInternalConfig.DEFAULT_BULKINSERT_ARE_PARTITIONER_RECORDS_SORTED)));
// 1st arg to createHooodieConfig is not really reuqired to be set. but passing it anyways.
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA.key()), path, tblName, properties);
HoodieWriteConfig config = DataSourceUtils.createHoodieConfig(properties.get(HoodieWriteConfig.AVRO_SCHEMA_STRING.key()), path, tblName, properties);
return new HoodieDataSourceInternalTable(instantTime, config, schema, getSparkSession(),
getConfiguration(), properties, populateMetaFields, arePartitionRecordsSorted);
}