1
0

[HUDI-1040] Make Hudi support Spark 3 (#2208)

* Fix flaky MOR unit test

* Update Spark APIs to make it be compatible with both spark2 & spark3

* Refactor bulk insert v2 part to make Hudi be able to compile with Spark3

* Add spark3 profile to handle fasterxml & spark version

* Create hudi-spark-common module & refactor hudi-spark related modules

Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
wenningd
2020-12-09 15:52:23 -08:00
committed by GitHub
parent 3a91d26d62
commit fce1453fa6
79 changed files with 1040 additions and 172 deletions

View File

@@ -0,0 +1,299 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi;
import org.apache.avro.generic.GenericRecord;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.HoodieWriteResult;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.CommitUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.common.util.TablePathUtils;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieNotSupportedException;
import org.apache.hudi.exception.TableNotFoundException;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser;
import org.apache.hudi.table.BulkInsertPartitioner;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
/**
* Utilities used throughout the data source.
*/
public class DataSourceUtils {
private static final Logger LOG = LogManager.getLogger(DataSourceUtils.class);
public static String getTablePath(FileSystem fs, Path[] userProvidedPaths) throws IOException {
LOG.info("Getting table path..");
for (Path path : userProvidedPaths) {
try {
Option<Path> tablePath = TablePathUtils.getTablePath(fs, path);
if (tablePath.isPresent()) {
return tablePath.get().toString();
}
} catch (HoodieException he) {
LOG.warn("Error trying to get table path from " + path.toString(), he);
}
}
throw new TableNotFoundException("Unable to find a hudi table for the user provided paths.");
}
/**
* Create a key generator class via reflection, passing in any configs needed.
* <p>
* If the class name of key generator is configured through the properties file, i.e., {@code props}, use the corresponding key generator class; otherwise, use the default key generator class
* specified in {@code DataSourceWriteOptions}.
*/
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
String keyGeneratorClass = props.getString(DataSourceWriteOptions.KEYGENERATOR_CLASS_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_KEYGENERATOR_CLASS_OPT_VAL());
try {
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
} catch (Throwable e) {
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
}
}
/**
* Create a date time parser class for TimestampBasedKeyGenerator, passing in any configs needed.
*/
public static AbstractHoodieDateTimeParser createDateTimeParser(TypedProperties props, String parserClass) throws IOException {
try {
return (AbstractHoodieDateTimeParser) ReflectionUtils.loadClass(parserClass, props);
} catch (Throwable e) {
throw new IOException("Could not load date time parser class " + parserClass, e);
}
}
/**
* Create a UserDefinedBulkInsertPartitioner class via reflection,
* <br>
* if the class name of UserDefinedBulkInsertPartitioner is configured through the HoodieWriteConfig.
*
* @see HoodieWriteConfig#getUserDefinedBulkInsertPartitionerClass()
*/
private static Option<BulkInsertPartitioner> createUserDefinedBulkInsertPartitioner(HoodieWriteConfig config)
throws HoodieException {
String bulkInsertPartitionerClass = config.getUserDefinedBulkInsertPartitionerClass();
try {
return StringUtils.isNullOrEmpty(bulkInsertPartitionerClass)
? Option.empty() :
Option.of((BulkInsertPartitioner) ReflectionUtils.loadClass(bulkInsertPartitionerClass));
} catch (Throwable e) {
throw new HoodieException("Could not create UserDefinedBulkInsertPartitioner class " + bulkInsertPartitionerClass, e);
}
}
/**
* Create a payload class via reflection, passing in an ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record, Comparable orderingVal)
throws IOException {
try {
return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[] {GenericRecord.class, Comparable.class}, record, orderingVal);
} catch (Throwable e) {
throw new IOException("Could not create payload for class: " + payloadClass, e);
}
}
/**
* Create a payload class via reflection, do not ordering/precombine value.
*/
public static HoodieRecordPayload createPayload(String payloadClass, GenericRecord record)
throws IOException {
try {
return (HoodieRecordPayload) ReflectionUtils.loadClass(payloadClass,
new Class<?>[] {Option.class}, Option.of(record));
} catch (Throwable e) {
throw new IOException("Could not create payload for class: " + payloadClass, e);
}
}
public static void checkRequiredProperties(TypedProperties props, List<String> checkPropNames) {
checkPropNames.forEach(prop -> {
if (!props.containsKey(prop)) {
throw new HoodieNotSupportedException("Required property " + prop + " is missing");
}
});
}
public static HoodieWriteConfig createHoodieConfig(String schemaStr, String basePath,
String tblName, Map<String, String> parameters) {
boolean asyncCompact = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.ASYNC_COMPACT_ENABLE_OPT_KEY()));
boolean inlineCompact = !asyncCompact && parameters.get(DataSourceWriteOptions.TABLE_TYPE_OPT_KEY())
.equals(DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL());
// insert/bulk-insert combining to be true, if filtering for duplicates
boolean combineInserts = Boolean.parseBoolean(parameters.get(DataSourceWriteOptions.INSERT_DROP_DUPS_OPT_KEY()));
HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder()
.withPath(basePath).withAutoCommit(false).combineInput(combineInserts, true);
if (schemaStr != null) {
builder = builder.withSchema(schemaStr);
}
return builder.forTable(tblName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withPayloadClass(parameters.get(DataSourceWriteOptions.PAYLOAD_CLASS_OPT_KEY()))
.withInlineCompaction(inlineCompact).build())
// override above with Hoodie configs specified as options.
.withProps(parameters).build();
}
public static SparkRDDWriteClient createHoodieClient(JavaSparkContext jssc, String schemaStr, String basePath,
String tblName, Map<String, String> parameters) {
return new SparkRDDWriteClient<>(new HoodieSparkEngineContext(jssc), createHoodieConfig(schemaStr, basePath, tblName, parameters), true);
}
public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) {
if (operation == WriteOperationType.INSERT_OVERWRITE || operation == WriteOperationType.INSERT_OVERWRITE_TABLE) {
return HoodieTimeline.REPLACE_COMMIT_ACTION;
} else {
return CommitUtils.getCommitActionType(tableType);
}
}
public static HoodieWriteResult doWriteOperation(SparkRDDWriteClient client, JavaRDD<HoodieRecord> hoodieRecords,
String instantTime, WriteOperationType operation) throws HoodieException {
switch (operation) {
case BULK_INSERT:
Option<BulkInsertPartitioner> userDefinedBulkInsertPartitioner =
createUserDefinedBulkInsertPartitioner(client.getConfig());
return new HoodieWriteResult(client.bulkInsert(hoodieRecords, instantTime, userDefinedBulkInsertPartitioner));
case INSERT:
return new HoodieWriteResult(client.insert(hoodieRecords, instantTime));
case UPSERT:
return new HoodieWriteResult(client.upsert(hoodieRecords, instantTime));
case INSERT_OVERWRITE:
return client.insertOverwrite(hoodieRecords, instantTime);
case INSERT_OVERWRITE_TABLE:
return client.insertOverwriteTable(hoodieRecords, instantTime);
default:
throw new HoodieException("Not a valid operation type for doWriteOperation: " + operation.toString());
}
}
public static HoodieWriteResult doDeleteOperation(SparkRDDWriteClient client, JavaRDD<HoodieKey> hoodieKeys,
String instantTime) {
return new HoodieWriteResult(client.delete(hoodieKeys, instantTime));
}
public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey,
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr, orderingVal);
return new HoodieRecord<>(hKey, payload);
}
public static HoodieRecord createHoodieRecord(GenericRecord gr, HoodieKey hKey,
String payloadClass) throws IOException {
HoodieRecordPayload payload = DataSourceUtils.createPayload(payloadClass, gr);
return new HoodieRecord<>(hKey, payload);
}
/**
* Drop records already present in the dataset.
*
* @param jssc JavaSparkContext
* @param incomingHoodieRecords HoodieRecords to deduplicate
* @param writeConfig HoodieWriteConfig
*/
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
HoodieWriteConfig writeConfig) {
try {
HoodieReadClient client = new HoodieReadClient<>(new HoodieSparkEngineContext(jssc), writeConfig);
return client.tagLocation(incomingHoodieRecords)
.filter(r -> !((HoodieRecord<HoodieRecordPayload>) r).isCurrentLocationKnown());
} catch (TableNotFoundException e) {
// this will be executed when there is no hoodie table yet
// so no dups to drop
return incomingHoodieRecords;
}
}
@SuppressWarnings("unchecked")
public static JavaRDD<HoodieRecord> dropDuplicates(JavaSparkContext jssc, JavaRDD<HoodieRecord> incomingHoodieRecords,
Map<String, String> parameters) {
HoodieWriteConfig writeConfig =
HoodieWriteConfig.newBuilder().withPath(parameters.get("path")).withProps(parameters).build();
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
}
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) {
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY()));
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = basePath;
hiveSyncConfig.usePreApacheInputFormat =
props.getBoolean(DataSourceWriteOptions.HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY(),
Boolean.parseBoolean(DataSourceWriteOptions.DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL()));
hiveSyncConfig.databaseName = props.getString(DataSourceWriteOptions.HIVE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_DATABASE_OPT_VAL());
hiveSyncConfig.tableName = props.getString(DataSourceWriteOptions.HIVE_TABLE_OPT_KEY());
hiveSyncConfig.baseFileFormat = baseFileFormat;
hiveSyncConfig.hiveUser =
props.getString(DataSourceWriteOptions.HIVE_USER_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_USER_OPT_VAL());
hiveSyncConfig.hivePass =
props.getString(DataSourceWriteOptions.HIVE_PASS_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_PASS_OPT_VAL());
hiveSyncConfig.jdbcUrl =
props.getString(DataSourceWriteOptions.HIVE_URL_OPT_KEY(), DataSourceWriteOptions.DEFAULT_HIVE_URL_OPT_VAL());
hiveSyncConfig.partitionFields =
props.getStringList(DataSourceWriteOptions.HIVE_PARTITION_FIELDS_OPT_KEY(), ",", new ArrayList<>());
hiveSyncConfig.partitionValueExtractorClass =
props.getString(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY(),
SlashEncodedDayPartitionValueExtractor.class.getName());
hiveSyncConfig.useJdbc = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_USE_JDBC_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_USE_JDBC_OPT_VAL()));
hiveSyncConfig.autoCreateDatabase = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_AUTO_CREATE_DATABASE_OPT_KEY(),
DataSourceWriteOptions.DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY()));
hiveSyncConfig.skipROSuffix = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX(),
DataSourceWriteOptions.DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL()));
hiveSyncConfig.supportTimestamp = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_SUPPORT_TIMESTAMP(),
DataSourceWriteOptions.DEFAULT_HIVE_SUPPORT_TIMESTAMP()));
return hiveSyncConfig;
}
}

View File

@@ -0,0 +1,334 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi
import org.apache.hudi.common.model.HoodieTableType
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload
import org.apache.hudi.common.model.WriteOperationType
import org.apache.hudi.hive.HiveSyncTool
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor
import org.apache.hudi.keygen.SimpleKeyGenerator
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
import org.apache.log4j.LogManager
/**
* List of options that can be passed to the Hoodie datasource,
* in addition to the hoodie client configs
*/
/**
* Options supported for reading hoodie tables.
*/
object DataSourceReadOptions {
private val log = LogManager.getLogger(DataSourceReadOptions.getClass)
/**
* Whether data needs to be read, in
*
* 1) Snapshot mode (obtain latest view, based on row & columnar data)
* 2) incremental mode (new data since an instantTime)
* 3) Read Optimized mode (obtain latest view, based on columnar data)
*
* Default: snapshot
*/
val QUERY_TYPE_OPT_KEY = "hoodie.datasource.query.type"
val QUERY_TYPE_SNAPSHOT_OPT_VAL = "snapshot"
val QUERY_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
val QUERY_TYPE_INCREMENTAL_OPT_VAL = "incremental"
val DEFAULT_QUERY_TYPE_OPT_VAL: String = QUERY_TYPE_SNAPSHOT_OPT_VAL
/**
* For Snapshot query on merge on read table. Use this key to define the payload class.
*/
val REALTIME_MERGE_OPT_KEY = "hoodie.datasource.merge.type"
val REALTIME_SKIP_MERGE_OPT_VAL = "skip_merge"
val REALTIME_PAYLOAD_COMBINE_OPT_VAL = "payload_combine"
val DEFAULT_REALTIME_MERGE_OPT_VAL = REALTIME_PAYLOAD_COMBINE_OPT_VAL
val READ_PATHS_OPT_KEY = "hoodie.datasource.read.paths"
@Deprecated
val VIEW_TYPE_OPT_KEY = "hoodie.datasource.view.type"
@Deprecated
val VIEW_TYPE_READ_OPTIMIZED_OPT_VAL = "read_optimized"
@Deprecated
val VIEW_TYPE_INCREMENTAL_OPT_VAL = "incremental"
@Deprecated
val VIEW_TYPE_REALTIME_OPT_VAL = "realtime"
@Deprecated
val DEFAULT_VIEW_TYPE_OPT_VAL = VIEW_TYPE_READ_OPTIMIZED_OPT_VAL
/**
* This eases migration from old configs to new configs.
*/
def translateViewTypesToQueryTypes(optParams: Map[String, String]) : Map[String, String] = {
val translation = Map(VIEW_TYPE_READ_OPTIMIZED_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL,
VIEW_TYPE_INCREMENTAL_OPT_VAL -> QUERY_TYPE_INCREMENTAL_OPT_VAL,
VIEW_TYPE_REALTIME_OPT_VAL -> QUERY_TYPE_SNAPSHOT_OPT_VAL)
if (!optParams.contains(QUERY_TYPE_OPT_KEY)) {
if (optParams.contains(VIEW_TYPE_OPT_KEY)) {
log.warn(VIEW_TYPE_OPT_KEY + " is deprecated and will be removed in a later release. Please use " + QUERY_TYPE_OPT_KEY)
optParams ++ Map(QUERY_TYPE_OPT_KEY -> translation(optParams(VIEW_TYPE_OPT_KEY)))
} else {
optParams ++ Map(QUERY_TYPE_OPT_KEY -> DEFAULT_QUERY_TYPE_OPT_VAL)
}
} else {
optParams
}
}
/**
* Instant time to start incrementally pulling data from. The instanttime here need not
* necessarily correspond to an instant on the timeline. New data written with an
* `instant_time > BEGIN_INSTANTTIME` are fetched out. For e.g: '20170901080000' will get
* all new data written after Sep 1, 2017 08:00AM.
*
* Default: None (Mandatory in incremental mode)
*/
val BEGIN_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.begin.instanttime"
/**
* Instant time to limit incrementally fetched data to. New data written with an
* `instant_time <= END_INSTANTTIME` are fetched out.
*
* Default: latest instant (i.e fetches all new data since begin instant time)
*
*/
val END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.end.instanttime"
/**
* If use the end instant schema when incrementally fetched data to.
*
* Default: false (use latest instant schema)
*
*/
val INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_KEY = "hoodie.datasource.read.schema.use.end.instanttime"
val DEFAULT_INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME_OPT_VAL = "false"
/**
* For use-cases like DeltaStreamer which reads from Hoodie Incremental table and applies opaque map functions,
* filters appearing late in the sequence of transformations cannot be automatically pushed down.
* This option allows setting filters directly on Hoodie Source
*/
val PUSH_DOWN_INCR_FILTERS_OPT_KEY = "hoodie.datasource.read.incr.filters"
val DEFAULT_PUSH_DOWN_FILTERS_OPT_VAL = ""
/**
* For the use-cases like users only want to incremental pull from certain partitions instead of the full table.
* This option allows using glob pattern to directly filter on path.
*/
val INCR_PATH_GLOB_OPT_KEY = "hoodie.datasource.read.incr.path.glob"
val DEFAULT_INCR_PATH_GLOB_OPT_VAL = ""
}
/**
* Options supported for writing hoodie tables.
*/
object DataSourceWriteOptions {
private val log = LogManager.getLogger(DataSourceWriteOptions.getClass)
/**
* The write operation, that this write should do
*
* Default: upsert()
*/
val OPERATION_OPT_KEY = "hoodie.datasource.write.operation"
val BULK_INSERT_OPERATION_OPT_VAL = WriteOperationType.BULK_INSERT.value
val INSERT_OPERATION_OPT_VAL = WriteOperationType.INSERT.value
val UPSERT_OPERATION_OPT_VAL = WriteOperationType.UPSERT.value
val DELETE_OPERATION_OPT_VAL = WriteOperationType.DELETE.value
val BOOTSTRAP_OPERATION_OPT_VAL = WriteOperationType.BOOTSTRAP.value
val INSERT_OVERWRITE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE.value
val INSERT_OVERWRITE_TABLE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE_TABLE.value
val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL
/**
* The table type for the underlying data, for this write.
* Note that this can't change across writes.
*
* Default: COPY_ON_WRITE
*/
val TABLE_TYPE_OPT_KEY = "hoodie.datasource.write.table.type"
val COW_TABLE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
val MOR_TABLE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
val DEFAULT_TABLE_TYPE_OPT_VAL = COW_TABLE_TYPE_OPT_VAL
@Deprecated
val STORAGE_TYPE_OPT_KEY = "hoodie.datasource.write.storage.type"
@Deprecated
val COW_STORAGE_TYPE_OPT_VAL = HoodieTableType.COPY_ON_WRITE.name
@Deprecated
val MOR_STORAGE_TYPE_OPT_VAL = HoodieTableType.MERGE_ON_READ.name
@Deprecated
val DEFAULT_STORAGE_TYPE_OPT_VAL = COW_STORAGE_TYPE_OPT_VAL
def translateStorageTypeToTableType(optParams: Map[String, String]) : Map[String, String] = {
if (optParams.contains(STORAGE_TYPE_OPT_KEY) && !optParams.contains(TABLE_TYPE_OPT_KEY)) {
log.warn(STORAGE_TYPE_OPT_KEY + " is deprecated and will be removed in a later release; Please use " + TABLE_TYPE_OPT_KEY)
optParams ++ Map(TABLE_TYPE_OPT_KEY -> optParams(STORAGE_TYPE_OPT_KEY))
} else {
optParams
}
}
/**
* Hive table name, to register the table into.
*
* Default: None (mandatory)
*/
val TABLE_NAME_OPT_KEY = "hoodie.datasource.write.table.name"
/**
* Field used in preCombining before actual write. When two records have the same
* key value, we will pick the one with the largest value for the precombine field,
* determined by Object.compareTo(..)
*/
val PRECOMBINE_FIELD_OPT_KEY = "hoodie.datasource.write.precombine.field"
val DEFAULT_PRECOMBINE_FIELD_OPT_VAL = "ts"
/**
* 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_OPT_KEY = "hoodie.datasource.write.payload.class"
val DEFAULT_PAYLOAD_OPT_VAL = classOf[OverwriteWithLatestAvroPayload].getName
/**
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
* will be obtained by invoking .toString() on the field value. Nested fields can be specified using
* the dot notation eg: `a.b.c`
*
*/
val RECORDKEY_FIELD_OPT_KEY = KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY
val DEFAULT_RECORDKEY_FIELD_OPT_VAL = "uuid"
/**
* Partition path field. Value to be used at the `partitionPath` component of `HoodieKey`. Actual
* value obtained by invoking .toString()
*/
val PARTITIONPATH_FIELD_OPT_KEY = KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY
val DEFAULT_PARTITIONPATH_FIELD_OPT_VAL = "partitionpath"
/**
* Flag to indicate whether to use Hive style partitioning.
* If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.
* By default false (the names of partition folders are only partition values)
*/
val HIVE_STYLE_PARTITIONING_OPT_KEY = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY
val DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = KeyGeneratorOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL
val URL_ENCODE_PARTITIONING_OPT_KEY = KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY
val DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL
/**
* Key generator class, that implements will extract the key out of incoming record
*
*/
val KEYGENERATOR_CLASS_OPT_KEY = "hoodie.datasource.write.keygenerator.class"
val DEFAULT_KEYGENERATOR_CLASS_OPT_VAL = classOf[SimpleKeyGenerator].getName
/**
* When set to true, will perform write operations directly using the spark native `Row` representation.
* By default, false (will be enabled as default in a future release)
*/
val ENABLE_ROW_WRITER_OPT_KEY = "hoodie.datasource.write.row.writer.enable"
val DEFAULT_ENABLE_ROW_WRITER_OPT_VAL = "false"
/**
* Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata.
* This is useful to store checkpointing information, in a consistent way with the hoodie timeline
*/
val COMMIT_METADATA_KEYPREFIX_OPT_KEY = "hoodie.datasource.write.commitmeta.key.prefix"
val DEFAULT_COMMIT_METADATA_KEYPREFIX_OPT_VAL = "_"
/**
* Flag to indicate whether to drop duplicates upon insert.
* By default insert will accept duplicates, to gain extra performance.
*/
val INSERT_DROP_DUPS_OPT_KEY = "hoodie.datasource.write.insert.drop.duplicates"
val DEFAULT_INSERT_DROP_DUPS_OPT_VAL = "false"
/**
* Flag to indicate how many times streaming job should retry for a failed microbatch
* By default 3
*/
val STREAMING_RETRY_CNT_OPT_KEY = "hoodie.datasource.write.streaming.retry.count"
val DEFAULT_STREAMING_RETRY_CNT_OPT_VAL = "3"
/**
* Flag to indicate how long (by millisecond) before a retry should issued for failed microbatch
* By default 2000 and it will be doubled by every retry
*/
val STREAMING_RETRY_INTERVAL_MS_OPT_KEY = "hoodie.datasource.write.streaming.retry.interval.ms"
val DEFAULT_STREAMING_RETRY_INTERVAL_MS_OPT_VAL = "2000"
/**
* Flag to indicate whether to ignore any non exception error (e.g. writestatus error)
* within a streaming microbatch
* By default true (in favor of streaming progressing over data integrity)
*/
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = "hoodie.datasource.write.streaming.ignore.failed.batch"
val DEFAULT_STREAMING_IGNORE_FAILED_BATCH_OPT_VAL = "true"
val META_SYNC_CLIENT_TOOL_CLASS = "hoodie.meta.sync.client.tool.class"
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = classOf[HiveSyncTool].getName
// HIVE SYNC SPECIFIC CONFIGS
//NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
// unexpected issues with config getting reset
val HIVE_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.hive_sync.enable"
val META_SYNC_ENABLED_OPT_KEY = "hoodie.datasource.meta.sync.enable"
val HIVE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.database"
val HIVE_TABLE_OPT_KEY = "hoodie.datasource.hive_sync.table"
val HIVE_BASE_FILE_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.base_file_format"
val HIVE_USER_OPT_KEY = "hoodie.datasource.hive_sync.username"
val HIVE_PASS_OPT_KEY = "hoodie.datasource.hive_sync.password"
val HIVE_URL_OPT_KEY = "hoodie.datasource.hive_sync.jdbcurl"
val HIVE_PARTITION_FIELDS_OPT_KEY = "hoodie.datasource.hive_sync.partition_fields"
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = "hoodie.datasource.hive_sync.partition_extractor_class"
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = "hoodie.datasource.hive_sync.assume_date_partitioning"
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = "hoodie.datasource.hive_sync.use_pre_apache_input_format"
val HIVE_USE_JDBC_OPT_KEY = "hoodie.datasource.hive_sync.use_jdbc"
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = "hoodie.datasource.hive_sync.auto_create_database"
val HIVE_SKIP_RO_SUFFIX = "hoodie.datasource.hive_sync.skip_ro_suffix"
val HIVE_SUPPORT_TIMESTAMP = "hoodie.datasource.hive_sync.support_timestamp"
// DEFAULT FOR HIVE SPECIFIC CONFIGS
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = "false"
val DEFAULT_HIVE_DATABASE_OPT_VAL = "default"
val DEFAULT_HIVE_TABLE_OPT_VAL = "unknown"
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = "PARQUET"
val DEFAULT_HIVE_USER_OPT_VAL = "hive"
val DEFAULT_HIVE_PASS_OPT_VAL = "hive"
val DEFAULT_HIVE_URL_OPT_VAL = "jdbc:hive2://localhost:10000"
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = ""
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = "false"
val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false"
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = "true"
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = "true"
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = "false"
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = "false"
// Async Compaction - Enabled by default for MOR
val ASYNC_COMPACT_ENABLE_OPT_KEY = "hoodie.datasource.compaction.async.enable"
val DEFAULT_ASYNC_COMPACT_ENABLE_OPT_VAL = "true"
}