1
0

[HUDI-2538] persist some configs to hoodie.properties when the first write (#3823)

This commit is contained in:
Yann Byron
2021-11-03 10:04:23 +08:00
committed by GitHub
parent 1f17467f73
commit 6351e5f4d0
25 changed files with 544 additions and 207 deletions

View File

@@ -21,10 +21,11 @@ package org.apache.hudi.table.upgrade;
import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.metadata.HoodieTableMetadataUtil; import org.apache.hudi.metadata.HoodieTableMetadataUtil;
import java.util.Collections; import java.util.HashMap;
import java.util.Map; import java.util.Map;
/** /**
@@ -39,6 +40,10 @@ public class TwoToThreeUpgradeHandler implements UpgradeHandler {
// table has been updated and is not backward compatible. // table has been updated and is not backward compatible.
HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context); HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context);
} }
return Collections.emptyMap(); Map<ConfigProperty, String> tablePropsToAdd = new HashMap<>();
tablePropsToAdd.put(HoodieTableConfig.URL_ENCODE_PARTITIONING, config.getStringOrDefault(HoodieTableConfig.URL_ENCODE_PARTITIONING));
tablePropsToAdd.put(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, config.getStringOrDefault(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
tablePropsToAdd.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, config.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
return tablePropsToAdd;
} }
} }

View File

@@ -19,14 +19,13 @@
package org.apache.hudi.keygen.factory; package org.apache.hudi.keygen.factory;
import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.util.ReflectionUtils;
import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieKeyGeneratorException; import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.keygen.BuiltinKeyGenerator;
import org.apache.hudi.keygen.ComplexKeyGenerator; import org.apache.hudi.keygen.ComplexKeyGenerator;
import org.apache.hudi.keygen.CustomKeyGenerator; import org.apache.hudi.keygen.CustomKeyGenerator;
import org.apache.hudi.keygen.GlobalDeleteKeyGenerator; import org.apache.hudi.keygen.GlobalDeleteKeyGenerator;
import org.apache.hudi.keygen.KeyGenUtils;
import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.KeyGenerator;
import org.apache.hudi.keygen.NonpartitionedKeyGenerator; import org.apache.hudi.keygen.NonpartitionedKeyGenerator;
import org.apache.hudi.keygen.SimpleKeyGenerator; import org.apache.hudi.keygen.SimpleKeyGenerator;
@@ -37,8 +36,9 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.util.HashMap;
import java.util.Locale; import java.util.Locale;
import java.util.Objects; import java.util.Map;
/** /**
* Factory help to create {@link org.apache.hudi.keygen.KeyGenerator}. * Factory help to create {@link org.apache.hudi.keygen.KeyGenerator}.
@@ -50,23 +50,37 @@ public class HoodieSparkKeyGeneratorFactory {
private static final Logger LOG = LoggerFactory.getLogger(HoodieSparkKeyGeneratorFactory.class); private static final Logger LOG = LoggerFactory.getLogger(HoodieSparkKeyGeneratorFactory.class);
private static final Map<String, String> COMMON_TO_SPARK_KEYGENERATOR = new HashMap<>();
static {
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.ComplexAvroKeyGenerator",
"org.apache.hudi.keygen.ComplexKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.CustomAvroKeyGenerator",
"org.apache.hudi.keygen.CustomKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.GlobalAvroDeleteKeyGenerator",
"org.apache.hudi.keygen.GlobalDeleteKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.NonpartitionedAvroKeyGenerator",
"org.apache.hudi.keygen.NonpartitionedKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.SimpleAvroKeyGenerator",
"org.apache.hudi.keygen.SimpleKeyGenerator");
COMMON_TO_SPARK_KEYGENERATOR.put("org.apache.hudi.keygen.TimestampBasedAvroKeyGenerator",
"org.apache.hudi.keygen.TimestampBasedKeyGenerator");
}
public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException { public static KeyGenerator createKeyGenerator(TypedProperties props) throws IOException {
// keyGenerator class name has higher priority String keyGeneratorClass = getKeyGeneratorClassName(props);
KeyGenerator keyGenerator = KeyGenUtils.createKeyGeneratorByClassName(props); try {
return (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
return Objects.isNull(keyGenerator) ? createKeyGeneratorByType(props) : keyGenerator; } catch (Throwable e) {
throw new IOException("Could not load key generator class " + keyGeneratorClass, e);
}
} }
private static BuiltinKeyGenerator createKeyGeneratorByType(TypedProperties props) throws IOException { public static String getKeyGeneratorClassName(TypedProperties props) {
// Use KeyGeneratorType.SIMPLE as default keyGeneratorType String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key(), null);
String keyGeneratorType =
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), null);
if (StringUtils.isNullOrEmpty(keyGeneratorType)) { if (StringUtils.isNullOrEmpty(keyGeneratorClass)) {
String keyGeneratorType = props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE.key(), KeyGeneratorType.SIMPLE.name());
LOG.info("The value of {} is empty, use SIMPLE", HoodieWriteConfig.KEYGENERATOR_TYPE.key()); LOG.info("The value of {} is empty, use SIMPLE", HoodieWriteConfig.KEYGENERATOR_TYPE.key());
keyGeneratorType = KeyGeneratorType.SIMPLE.name();
}
KeyGeneratorType keyGeneratorTypeEnum; KeyGeneratorType keyGeneratorTypeEnum;
try { try {
keyGeneratorTypeEnum = KeyGeneratorType.valueOf(keyGeneratorType.toUpperCase(Locale.ROOT)); keyGeneratorTypeEnum = KeyGeneratorType.valueOf(keyGeneratorType.toUpperCase(Locale.ROOT));
@@ -75,20 +89,34 @@ public class HoodieSparkKeyGeneratorFactory {
} }
switch (keyGeneratorTypeEnum) { switch (keyGeneratorTypeEnum) {
case SIMPLE: case SIMPLE:
return new SimpleKeyGenerator(props); keyGeneratorClass = SimpleKeyGenerator.class.getName();
break;
case COMPLEX: case COMPLEX:
return new ComplexKeyGenerator(props); keyGeneratorClass = ComplexKeyGenerator.class.getName();
break;
case TIMESTAMP: case TIMESTAMP:
return new TimestampBasedKeyGenerator(props); keyGeneratorClass = TimestampBasedKeyGenerator.class.getName();
break;
case CUSTOM: case CUSTOM:
return new CustomKeyGenerator(props); keyGeneratorClass = CustomKeyGenerator.class.getName();
break;
case NON_PARTITION: case NON_PARTITION:
return new NonpartitionedKeyGenerator(props); keyGeneratorClass = NonpartitionedKeyGenerator.class.getName();
break;
case GLOBAL_DELETE: case GLOBAL_DELETE:
return new GlobalDeleteKeyGenerator(props); keyGeneratorClass = GlobalDeleteKeyGenerator.class.getName();
break;
default: default:
throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType); throw new HoodieKeyGeneratorException("Unsupported keyGenerator Type " + keyGeneratorType);
} }
} }
return keyGeneratorClass;
}
/**
* Convert hoodie-common KeyGenerator to SparkKeyGeneratorInterface implement.
*/
public static String convertToSparkKeyGenerator(String keyGeneratorClassName) {
return COMMON_TO_SPARK_KEYGENERATOR.getOrDefault(keyGeneratorClassName, keyGeneratorClassName);
}
} }

View File

@@ -22,6 +22,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
@@ -33,6 +34,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.config.metrics.HoodieMetricsConfig; import org.apache.hudi.config.metrics.HoodieMetricsConfig;
import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig; import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig;
import org.apache.hudi.index.HoodieIndex; import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter; import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter; import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
@@ -50,6 +52,7 @@ import java.io.IOException;
import java.util.Arrays; import java.util.Arrays;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Properties;
import static java.util.Arrays.asList; import static java.util.Arrays.asList;
import static java.util.Collections.emptyList; import static java.util.Collections.emptyList;
@@ -268,6 +271,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata,
boolean enableMetrics, boolean enableFullScan) { boolean enableMetrics, boolean enableFullScan) {
Properties properties = new Properties();
properties.put(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), SimpleKeyGenerator.class.getName());
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA) return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2) .withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2)
.withAutoCommit(autoCommit) .withAutoCommit(autoCommit)
@@ -287,7 +292,8 @@ public class TestHoodieMetadataBase extends HoodieClientTestHarness {
.withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics) .withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics)
.withExecutorMetrics(true).build()) .withExecutorMetrics(true).build())
.withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder() .withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder()
.usePrefix("unit-test").build()); .usePrefix("unit-test").build())
.withProperties(properties);
} }
} }

View File

@@ -33,6 +33,8 @@ import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Test; import org.junit.jupiter.api.Test;
import java.io.IOException; import java.io.IOException;
import java.io.PrintWriter;
import java.io.StringWriter;
public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities { public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
@@ -122,6 +124,13 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
return properties; return properties;
} }
private String stackTraceToString(Throwable e) {
StringWriter sw = new StringWriter();
PrintWriter pw = new PrintWriter(sw);
e.printStackTrace(pw);
return sw.toString();
}
@Test @Test
public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException { public void testSimpleKeyGeneratorWithKeyGeneratorClass() throws IOException {
testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true)); testSimpleKeyGenerator(getPropertiesForSimpleKeyGen(true));
@@ -259,7 +268,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
.getMessage() .getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found")); .contains("Property hoodie.datasource.write.recordkey.field not found"));
} else { } else {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found")); Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
} }
} }
@@ -282,7 +291,7 @@ public class TestCustomKeyGenerator extends KeyGeneratorTestUtilities {
.getMessage() .getMessage()
.contains("Property hoodie.datasource.write.recordkey.field not found")); .contains("Property hoodie.datasource.write.recordkey.field not found"));
} else { } else {
Assertions.assertTrue(e.getMessage().contains("Property hoodie.datasource.write.recordkey.field not found")); Assertions.assertTrue(stackTraceToString(e).contains("Property hoodie.datasource.write.recordkey.field not found"));
} }
} }
} }

View File

@@ -249,7 +249,15 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
initMetaClient(getTableType()); initMetaClient(getTableType());
} }
protected void initMetaClient(Properties properties) throws IOException {
initMetaClient(getTableType(), properties);
}
protected void initMetaClient(HoodieTableType tableType) throws IOException { protected void initMetaClient(HoodieTableType tableType) throws IOException {
initMetaClient(tableType, new Properties());
}
protected void initMetaClient(HoodieTableType tableType, Properties properties) throws IOException {
if (basePath == null) { if (basePath == null) {
throw new IllegalStateException("The base path has not been initialized."); throw new IllegalStateException("The base path has not been initialized.");
} }
@@ -258,7 +266,10 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
throw new IllegalStateException("The Spark context has not been initialized."); throw new IllegalStateException("The Spark context has not been initialized.");
} }
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType); if (tableName != null && !tableName.isEmpty()) {
properties.put(HoodieTableConfig.NAME.key(), tableName);
}
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType, properties);
} }
protected Properties getPropertiesForKeyGen() { protected Properties getPropertiesForKeyGen() {

View File

@@ -74,6 +74,10 @@ public class HoodieConfig implements Serializable {
} }
} }
public Boolean contains(String key) {
return props.containsKey(key);
}
public <T> boolean contains(ConfigProperty<T> configProperty) { public <T> boolean contains(ConfigProperty<T> configProperty) {
if (props.containsKey(configProperty.key())) { if (props.containsKey(configProperty.key())) {
return true; return true;
@@ -135,7 +139,7 @@ public class HoodieConfig implements Serializable {
public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty) { public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty) {
Option<Object> rawValue = getRawValue(configProperty); Option<Object> rawValue = getRawValue(configProperty);
return rawValue.map(v -> Boolean.parseBoolean(v.toString())) return rawValue.map(v -> Boolean.parseBoolean(v.toString()))
.orElse((Boolean) configProperty.defaultValue()); .orElse(Boolean.parseBoolean(configProperty.defaultValue().toString()));
} }
public <T> Long getLong(ConfigProperty<T> configProperty) { public <T> Long getLong(ConfigProperty<T> configProperty) {

View File

@@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.avro.Schema; import org.apache.avro.Schema;
import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataInputStream;
@@ -161,6 +162,9 @@ public class HoodieTableConfig extends HoodieConfig {
.noDefaultValue() .noDefaultValue()
.withDocumentation("Key Generator class property for the hoodie table"); .withDocumentation("Key Generator class property for the hoodie table");
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName(); public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) { public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
@@ -363,6 +367,18 @@ public class HoodieTableConfig extends HoodieConfig {
return getString(RECORDKEY_FIELDS); return getString(RECORDKEY_FIELDS);
} }
public String getKeyGeneratorClassName() {
return getString(KEY_GENERATOR_CLASS_NAME);
}
public String getHiveStylePartitioningEnable() {
return getString(HIVE_STYLE_PARTITIONING_ENABLE);
}
public String getUrlEncodePartitoning() {
return getString(URL_ENCODE_PARTITIONING);
}
public Map<String, String> propsMap() { public Map<String, String> propsMap() {
return props.entrySet().stream() return props.entrySet().stream()
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue()))); .collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));

View File

@@ -637,6 +637,8 @@ public class HoodieTableMetaClient implements Serializable {
private Boolean bootstrapIndexEnable; private Boolean bootstrapIndexEnable;
private Boolean populateMetaFields; private Boolean populateMetaFields;
private String keyGeneratorClassProp; private String keyGeneratorClassProp;
private Boolean hiveStylePartitioningEnable;
private Boolean urlEncodePartitioning;
private PropertyBuilder() { private PropertyBuilder() {
@@ -725,6 +727,16 @@ public class HoodieTableMetaClient implements Serializable {
return this; return this;
} }
public PropertyBuilder setHiveStylePartitioningEnable(Boolean hiveStylePartitioningEnable) {
this.hiveStylePartitioningEnable = hiveStylePartitioningEnable;
return this;
}
public PropertyBuilder setUrlEncodePartitioning(Boolean urlEncodePartitioning) {
this.urlEncodePartitioning = urlEncodePartitioning;
return this;
}
public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
return setTableType(metaClient.getTableType()) return setTableType(metaClient.getTableType())
.setTableName(metaClient.getTableConfig().getTableName()) .setTableName(metaClient.getTableConfig().getTableName())
@@ -786,6 +798,12 @@ public class HoodieTableMetaClient implements Serializable {
if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) { if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) {
setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)); setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
} }
if (hoodieConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE)) {
setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE));
}
if (hoodieConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING)) {
setUrlEncodePartitioning(hoodieConfig.getBoolean(HoodieTableConfig.URL_ENCODE_PARTITIONING));
}
return this; return this;
} }
@@ -849,6 +867,12 @@ public class HoodieTableMetaClient implements Serializable {
if (null != keyGeneratorClassProp) { if (null != keyGeneratorClassProp) {
tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGeneratorClassProp); tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGeneratorClassProp);
} }
if (null != hiveStylePartitioningEnable) {
tableConfig.setValue(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE, Boolean.toString(hiveStylePartitioningEnable));
}
if (null != urlEncodePartitioning) {
tableConfig.setValue(HoodieTableConfig.URL_ENCODE_PARTITIONING, Boolean.toString(urlEncodePartitioning));
}
return tableConfig.getProps(); return tableConfig.getProps();
} }

View File

@@ -34,12 +34,17 @@ import java.io.IOException;
*/ */
public class HoodieCommonTestHarness { public class HoodieCommonTestHarness {
protected String tableName = null;
protected String basePath = null; protected String basePath = null;
protected transient HoodieTestDataGenerator dataGen = null; protected transient HoodieTestDataGenerator dataGen = null;
protected transient HoodieTableMetaClient metaClient; protected transient HoodieTableMetaClient metaClient;
@TempDir @TempDir
public java.nio.file.Path tempDir; public java.nio.file.Path tempDir;
protected void setTableName(String tableName) {
this.tableName = tableName;
}
/** /**
* Initializes basePath. * Initializes basePath.
*/ */

View File

@@ -154,14 +154,12 @@ class DefaultSource extends RelationProvider
mode: SaveMode, mode: SaveMode,
optParams: Map[String, String], optParams: Map[String, String],
df: DataFrame): BaseRelation = { df: DataFrame): BaseRelation = {
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*) val dfWithoutMetaCols = df.drop(HoodieRecord.HOODIE_META_COLUMNS.asScala:_*)
if (translatedOptions(OPERATION.key).equals(BOOTSTRAP_OPERATION_OPT_VAL)) { if (optParams.get(OPERATION.key).contains(BOOTSTRAP_OPERATION_OPT_VAL)) {
HoodieSparkSqlWriter.bootstrap(sqlContext, mode, translatedOptions, dfWithoutMetaCols) HoodieSparkSqlWriter.bootstrap(sqlContext, mode, optParams, dfWithoutMetaCols)
} else { } else {
HoodieSparkSqlWriter.write(sqlContext, mode, translatedOptions, dfWithoutMetaCols) HoodieSparkSqlWriter.write(sqlContext, mode, optParams, dfWithoutMetaCols)
} }
new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema) new HoodieEmptyRelation(sqlContext, dfWithoutMetaCols.schema)
} }
@@ -170,11 +168,9 @@ class DefaultSource extends RelationProvider
optParams: Map[String, String], optParams: Map[String, String],
partitionColumns: Seq[String], partitionColumns: Seq[String],
outputMode: OutputMode): Sink = { outputMode: OutputMode): Sink = {
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(optParams)
val translatedOptions = DataSourceWriteOptions.translateSqlOptions(parameters)
new HoodieStreamingSink( new HoodieStreamingSink(
sqlContext, sqlContext,
translatedOptions, optParams,
partitionColumns, partitionColumns,
outputMode) outputMode)
} }

View File

@@ -17,13 +17,13 @@
package org.apache.hudi package org.apache.hudi
import org.apache.avro.Schema import org.apache.avro.Schema
import org.apache.avro.generic.GenericRecord import org.apache.avro.generic.GenericRecord
import org.apache.hadoop.conf.Configuration import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.DataSourceOptionsHelper.{allAlternatives, translateConfigurations}
import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient} import org.apache.hudi.client.{HoodieWriteResult, SparkRDDWriteClient}
import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.config.{HoodieConfig, HoodieMetadataConfig, TypedProperties}
@@ -31,12 +31,13 @@ import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType}
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils} import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils}
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
import org.apache.hudi.exception.HoodieException import org.apache.hudi.exception.HoodieException
import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows} import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRowsFactory, NonSortPartitionerWithRows}
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool} import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
import org.apache.hudi.index.SparkHoodieIndexFactory
import org.apache.hudi.internal.DataSourceInternalWriterHelper import org.apache.hudi.internal.DataSourceInternalWriterHelper
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.hudi.sync.common.AbstractSyncTool
@@ -51,9 +52,9 @@ import org.apache.spark.{SPARK_VERSION, SparkContext}
import java.util import java.util
import java.util.Properties import java.util.Properties
import org.apache.hudi.index.SparkHoodieIndexFactory
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.mutable
import scala.collection.mutable.StringBuilder
import scala.collection.mutable.ListBuffer import scala.collection.mutable.ListBuffer
object HoodieSparkSqlWriter { object HoodieSparkSqlWriter {
@@ -65,7 +66,7 @@ object HoodieSparkSqlWriter {
def write(sqlContext: SQLContext, def write(sqlContext: SQLContext,
mode: SaveMode, mode: SaveMode,
parameters: Map[String, String], optParams: Map[String, String],
df: DataFrame, df: DataFrame,
hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty, hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty, hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty,
@@ -75,16 +76,23 @@ object HoodieSparkSqlWriter {
: (Boolean, common.util.Option[String], common.util.Option[String], common.util.Option[String], : (Boolean, common.util.Option[String], common.util.Option[String], common.util.Option[String],
SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = { SparkRDDWriteClient[HoodieRecordPayload[Nothing]], HoodieTableConfig) = {
assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set")
val path = optParams("path")
val basePath = new Path(path)
val sparkContext = sqlContext.sparkContext val sparkContext = sqlContext.sparkContext
val path = parameters.get("path") val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters) tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
val tblNameOp = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.") var tableConfig = getHoodieTableConfig(sparkContext, path, hoodieTableConfigOpt)
validateTableConfig(sqlContext.sparkSession, optParams, tableConfig)
val (parameters, hoodieConfig) = mergeParamsAndGetHoodieConfig(optParams, tableConfig)
val tblName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME,
s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.").trim
assert(!StringUtils.isNullOrEmpty(hoodieConfig.getString(HoodieWriteConfig.TBL_NAME)),
s"'${HoodieWriteConfig.TBL_NAME.key}' must be set.")
asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined asyncCompactionTriggerFnDefined = asyncCompactionTriggerFn.isDefined
asyncClusteringTriggerFnDefined = asyncClusteringTriggerFn.isDefined asyncClusteringTriggerFnDefined = asyncClusteringTriggerFn.isDefined
if (path.isEmpty) {
throw new HoodieException(s"'path' must be set.")
}
val tblName = tblNameOp.trim
sparkContext.getConf.getOption("spark.serializer") match { sparkContext.getConf.getOption("spark.serializer") match {
case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") => case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") =>
case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer") case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer")
@@ -105,12 +113,8 @@ object HoodieSparkSqlWriter {
} }
val jsc = new JavaSparkContext(sparkContext) val jsc = new JavaSparkContext(sparkContext)
val basePath = new Path(path.get)
val instantTime = HoodieActiveTimeline.createNewInstantTime() val instantTime = HoodieActiveTimeline.createNewInstantTime()
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration) val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(new TypedProperties(hoodieConfig.getProps))
tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
var tableConfig = getHoodieTableConfig(sparkContext, path.get, hoodieTableConfigOpt)
val keyGenerator = HoodieSparkKeyGeneratorFactory.createKeyGenerator(toProperties(parameters))
if (mode == SaveMode.Ignore && tableExists) { if (mode == SaveMode.Ignore && tableExists) {
log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.") log.warn(s"hoodie table at $basePath already exists. Ignoring & not performing actual writes.")
@@ -124,7 +128,7 @@ object HoodieSparkSqlWriter {
val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT) val baseFileFormat = hoodieConfig.getStringOrDefault(HoodieTableConfig.BASE_FILE_FORMAT)
val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER) val archiveLogFolder = hoodieConfig.getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER)
val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD) val recordKeyFields = hoodieConfig.getString(DataSourceWriteOptions.RECORDKEY_FIELD)
val populateMetaFields = parameters.getOrElse(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()).toBoolean val populateMetaFields = hoodieConfig.getBooleanOrDefault(HoodieTableConfig.POPULATE_META_FIELDS)
val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder() val tableMetaClient = HoodieTableMetaClient.withPropertyBuilder()
.setTableType(tableType) .setTableType(tableType)
@@ -138,7 +142,9 @@ object HoodieSparkSqlWriter {
.setPopulateMetaFields(populateMetaFields) .setPopulateMetaFields(populateMetaFields)
.setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD)) .setRecordKeyFields(hoodieConfig.getString(RECORDKEY_FIELD))
.setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME)) .setKeyGeneratorClassProp(hoodieConfig.getString(KEYGENERATOR_CLASS_NAME))
.initTable(sparkContext.hadoopConfiguration, path.get) .setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING))
.setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING))
.initTable(sparkContext.hadoopConfiguration, path)
tableConfig = tableMetaClient.getTableConfig tableConfig = tableMetaClient.getTableConfig
} }
@@ -169,7 +175,7 @@ object HoodieSparkSqlWriter {
// Create a HoodieWriteClient & issue the delete. // Create a HoodieWriteClient & issue the delete.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
null, path.get, tblName, null, path, tblName,
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key))) mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
@@ -200,7 +206,7 @@ object HoodieSparkSqlWriter {
} }
// Create a HoodieWriteClient & issue the delete. // Create a HoodieWriteClient & issue the delete.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
null, path.get, tblName, null, path, tblName,
mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key))) mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)))
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] .asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
// Issue delete partitions // Issue delete partitions
@@ -244,7 +250,7 @@ object HoodieSparkSqlWriter {
val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema val writeSchema = if (dropPartitionColumns) generateSchemaWithoutPartitionColumns(partitionColumns, schema) else schema
// Create a HoodieWriteClient & issue the write. // Create a HoodieWriteClient & issue the write.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path.get, val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, writeSchema.toString, path,
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key) tblName, mapAsJavaMap(parameters - HoodieWriteConfig.AUTO_COMMIT_ENABLE.key)
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] )).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
@@ -326,14 +332,21 @@ object HoodieSparkSqlWriter {
def bootstrap(sqlContext: SQLContext, def bootstrap(sqlContext: SQLContext,
mode: SaveMode, mode: SaveMode,
parameters: Map[String, String], optParams: Map[String, String],
df: DataFrame, df: DataFrame,
hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty, hoodieTableConfigOpt: Option[HoodieTableConfig] = Option.empty,
hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty): Boolean = { hoodieWriteClient: Option[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]] = Option.empty): Boolean = {
assert(optParams.get("path").exists(!StringUtils.isNullOrEmpty(_)), "'path' must be set")
val path = optParams("path")
val basePath = new Path(path)
val sparkContext = sqlContext.sparkContext val sparkContext = sqlContext.sparkContext
val path = parameters.getOrElse("path", throw new HoodieException("'path' must be set.")) val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters) tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
var tableConfig = getHoodieTableConfig(sparkContext, path, hoodieTableConfigOpt)
validateTableConfig(sqlContext.sparkSession, optParams, tableConfig)
val (parameters, hoodieConfig) = mergeParamsAndGetHoodieConfig(optParams, tableConfig)
val tableName = hoodieConfig.getStringOrThrow(HoodieWriteConfig.TBL_NAME, s"'${HoodieWriteConfig.TBL_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 tableType = hoodieConfig.getStringOrDefault(TABLE_TYPE)
val bootstrapBasePath = hoodieConfig.getStringOrThrow(BASE_PATH, val bootstrapBasePath = hoodieConfig.getStringOrThrow(BASE_PATH,
@@ -349,10 +362,6 @@ object HoodieSparkSqlWriter {
schema = HoodieAvroUtils.getNullSchema.toString schema = HoodieAvroUtils.getNullSchema.toString
} }
val basePath = new Path(path)
val fs = basePath.getFileSystem(sparkContext.hadoopConfiguration)
tableExists = fs.exists(new Path(basePath, HoodieTableMetaClient.METAFOLDER_NAME))
val tableConfig = getHoodieTableConfig(sparkContext, path, hoodieTableConfigOpt)
// Handle various save modes // Handle various save modes
if (mode == SaveMode.Ignore && tableExists) { if (mode == SaveMode.Ignore && tableExists) {
@@ -381,6 +390,8 @@ object HoodieSparkSqlWriter {
.setPartitionFields(partitionColumns) .setPartitionFields(partitionColumns)
.setPopulateMetaFields(populateMetaFields) .setPopulateMetaFields(populateMetaFields)
.setKeyGeneratorClassProp(keyGenProp) .setKeyGeneratorClassProp(keyGenProp)
.setHiveStylePartitioningEnable(hoodieConfig.getBoolean(HIVE_STYLE_PARTITIONING))
.setUrlEncodePartitioning(hoodieConfig.getBoolean(URL_ENCODE_PARTITIONING))
.initTable(sparkContext.hadoopConfiguration, path) .initTable(sparkContext.hadoopConfiguration, path)
} }
@@ -401,7 +412,7 @@ object HoodieSparkSqlWriter {
df: DataFrame, df: DataFrame,
tblName: String, tblName: String,
basePath: Path, basePath: Path,
path: Option[String], path: String,
instantTime: String, instantTime: String,
partitionColumns: String): (Boolean, common.util.Option[String]) = { partitionColumns: String): (Boolean, common.util.Option[String]) = {
val sparkContext = sqlContext.sparkContext val sparkContext = sqlContext.sparkContext
@@ -424,7 +435,7 @@ object HoodieSparkSqlWriter {
throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet") throw new HoodieException("Dropping duplicates with bulk_insert in row writer path is not supported yet")
} }
val params = parameters.updated(HoodieWriteConfig.AVRO_SCHEMA_STRING.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 writeConfig = DataSourceUtils.createHoodieConfig(schema.toString, path, tblName, mapAsJavaMap(params))
val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) { val bulkInsertPartitionerRows : BulkInsertPartitioner[Dataset[Row]] = if (populateMetaFields) {
val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig) val userDefinedBulkInsertPartitionerOpt = DataSourceUtils.createUserDefinedBulkInsertPartitionerWithRows(writeConfig)
if (userDefinedBulkInsertPartitionerOpt.isPresent) { if (userDefinedBulkInsertPartitionerOpt.isPresent) {
@@ -699,4 +710,49 @@ object HoodieSparkSqlWriter {
null null
} }
} }
private def validateTableConfig(spark: SparkSession, params: Map[String, String],
tableConfig: HoodieTableConfig): Unit = {
val resolver = spark.sessionState.conf.resolver
val diffConfigs = StringBuilder.newBuilder
params.foreach { case (key, value) =>
val existingValue = getStringFromTableConfigWithAlternatives(tableConfig, key)
if (null != existingValue && !resolver(existingValue, value)) {
diffConfigs.append(s"$key:\t$value\t${tableConfig.getString(key)}\n")
}
}
if (diffConfigs.nonEmpty) {
diffConfigs.insert(0, "\nConfig conflict(key\tcurrent value\texisting value):\n")
throw new HoodieException(diffConfigs.toString.trim)
}
}
private def mergeParamsAndGetHoodieConfig(optParams: Map[String, String],
tableConfig: HoodieTableConfig): (Map[String, String], HoodieConfig) = {
val mergedParams = mutable.Map.empty ++
DataSourceWriteOptions.translateSqlOptions(HoodieWriterUtils.parametersWithWriteDefaults(optParams))
if (!mergedParams.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
&& mergedParams.contains(KEYGENERATOR_CLASS_NAME.key)) {
mergedParams(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = mergedParams(KEYGENERATOR_CLASS_NAME.key)
}
if (null != tableConfig) {
tableConfig.getProps.foreach { case (key, value) =>
mergedParams(key) = value
}
}
val params = mergedParams.toMap
(params, HoodieWriterUtils.convertMapToHoodieConfig(params))
}
private def getStringFromTableConfigWithAlternatives(tableConfig: HoodieTableConfig, key: String): String = {
if (null == tableConfig) {
null
} else {
if (allAlternatives.contains(key)) {
tableConfig.getString(allAlternatives(key))
} else {
tableConfig.getString(key)
}
}
}
} }

View File

@@ -48,9 +48,12 @@ class HoodieStreamingSink(sqlContext: SQLContext,
private val log = LogManager.getLogger(classOf[HoodieStreamingSink]) private val log = LogManager.getLogger(classOf[HoodieStreamingSink])
private val retryCnt = options(DataSourceWriteOptions.STREAMING_RETRY_CNT.key).toInt private val retryCnt = options.getOrDefault(DataSourceWriteOptions.STREAMING_RETRY_CNT.key,
private val retryIntervalMs = options(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS.key).toLong DataSourceWriteOptions.STREAMING_RETRY_CNT.defaultValue).toInt
private val ignoreFailedBatch = options(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key).toBoolean private val retryIntervalMs = options.getOrDefault(DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS.key,
DataSourceWriteOptions.STREAMING_RETRY_INTERVAL_MS.defaultValue).toLong
private val ignoreFailedBatch = options.getOrDefault(DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.key,
DataSourceWriteOptions.STREAMING_IGNORE_FAILED_BATCH.defaultValue).toBoolean
private var isAsyncCompactorServiceShutdownAbnormally = false private var isAsyncCompactorServiceShutdownAbnormally = false
private var isAsyncClusteringServiceShutdownAbnormally = false private var isAsyncClusteringServiceShutdownAbnormally = false

View File

@@ -92,10 +92,9 @@ object HoodieWriterUtils {
* @return * @return
*/ */
def getPartitionColumns(parameters: Map[String, String]): String = { def getPartitionColumns(parameters: Map[String, String]): String = {
val props = new TypedProperties() val props = new Properties()
props.putAll(parameters.asJava) props.putAll(parameters.asJava)
val keyGen = HoodieSparkKeyGeneratorFactory.createKeyGenerator(props) HoodieSparkUtils.getPartitionColumns(props)
HoodieSparkUtils.getPartitionColumns(keyGen, props)
} }
def convertMapToHoodieConfig(parameters: Map[String, String]): HoodieConfig = { def convertMapToHoodieConfig(parameters: Map[String, String]): HoodieConfig = {

View File

@@ -120,8 +120,13 @@ object HoodieOptionConfig {
*/ */
def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = { def mappingSqlOptionToTableConfig(options: Map[String, String]): Map[String, String] = {
defaultTableConfig ++ defaultTableConfig ++
options.filterKeys(k => keyTableConfigMapping.contains(k)) options.map { case (k, v) =>
.map(kv => keyTableConfigMapping(kv._1) -> valueMapping.getOrElse(kv._2, kv._2)) if (keyTableConfigMapping.contains(k)) {
keyTableConfigMapping(k) -> valueMapping.getOrElse(v, v)
} else {
k -> v
}
}
} }
/** /**

View File

@@ -41,8 +41,12 @@ import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOL
import org.apache.spark.sql.types.StructType import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
import org.apache.spark.{SPARK_VERSION, SparkConf} import org.apache.spark.{SPARK_VERSION, SparkConf}
import java.util.{Locale, Properties} import java.util.{Locale, Properties}
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import scala.collection.JavaConverters._ import scala.collection.JavaConverters._
import scala.collection.mutable import scala.collection.mutable
@@ -95,30 +99,8 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
// Get options from the external table and append with the options in ddl. // Get options from the external table and append with the options in ddl.
val originTableConfig = HoodieOptionConfig.mappingTableConfigToSqlOption( val originTableConfig = HoodieOptionConfig.mappingTableConfigToSqlOption(
metaClient.getTableConfig.getProps.asScala.toMap) metaClient.getTableConfig.getProps.asScala.toMap)
val extraConfig = extraTableConfig(sparkSession, isTableExists, originTableConfig)
val allPartitionPaths = getAllPartitionPaths(sparkSession, table) val options = originTableConfig ++ table.storage.properties ++ extraConfig
var upgrateConfig = Map.empty[String, String]
// If this is a non-hive-styled partition table, disable the hive style config.
// (By default this config is enable for spark sql)
upgrateConfig = if (!isHiveStyledPartitioning(allPartitionPaths, table)) {
upgrateConfig + (DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "false")
} else {
upgrateConfig
}
upgrateConfig = if (!isUrlEncodeEnabled(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.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 userSpecifiedSchema = table.schema val userSpecifiedSchema = table.schema
if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) { if (userSpecifiedSchema.isEmpty && tableSchema.isDefined) {
@@ -137,7 +119,8 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
s". The associated location('$path') already exists.") s". The associated location('$path') already exists.")
} }
// Add the meta fields to the schema if this is a managed table or an empty external table. // Add the meta fields to the schema if this is a managed table or an empty external table.
(addMetaFields(table.schema), table.storage.properties) val options = table.storage.properties ++ extraTableConfig(sparkSession, false)
(addMetaFields(table.schema), options)
} }
val tableType = HoodieOptionConfig.getTableType(table.storage.properties) val tableType = HoodieOptionConfig.getTableType(table.storage.properties)
@@ -314,6 +297,43 @@ case class CreateHoodieTableCommand(table: CatalogTable, ignoreIfExists: Boolean
s"'${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR}'") s"'${HoodieOptionConfig.SQL_VALUE_TABLE_TYPE_MOR}'")
} }
} }
def extraTableConfig(sparkSession: SparkSession, isTableExists: Boolean,
originTableConfig: Map[String, String] = Map.empty): Map[String, String] = {
val extraConfig = mutable.Map.empty[String, String]
if (isTableExists) {
val allPartitionPaths = getAllPartitionPaths(sparkSession, table)
if (originTableConfig.contains(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)) {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
originTableConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
} else {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) =
String.valueOf(isHiveStyledPartitioning(allPartitionPaths, table))
}
if (originTableConfig.contains(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)) {
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
originTableConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
} else {
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) =
String.valueOf(isUrlEncodeEnabled(allPartitionPaths, table))
}
} else {
extraConfig(HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key) = "true"
extraConfig(HoodieTableConfig.URL_ENCODE_PARTITIONING.key) = HoodieTableConfig.URL_ENCODE_PARTITIONING.defaultValue()
}
val primaryColumns = HoodieOptionConfig.getPrimaryColumns(originTableConfig ++ table.storage.properties)
if (primaryColumns.isEmpty) {
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[UuidKeyGenerator].getCanonicalName
} else if (originTableConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)) {
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) =
HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(
originTableConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key))
} else {
extraConfig(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key) = classOf[ComplexKeyGenerator].getCanonicalName
}
extraConfig.toMap
}
} }
object CreateHoodieTableCommand extends Logging { object CreateHoodieTableCommand extends Logging {
@@ -342,6 +362,9 @@ object CreateHoodieTableCommand extends Logging {
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PRECOMBINE_FIELD.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.PARTITION_FIELDS.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key) checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.RECORDKEY_FIELDS.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.URL_ENCODE_PARTITIONING.key)
checkTableConfigEqual(originTableConfig, tableOptions, HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE.key)
// Save all the table config to the hoodie.properties. // Save all the table config to the hoodie.properties.
val parameters = originTableConfig ++ tableOptions val parameters = originTableConfig ++ tableOptions
val properties = new Properties() val properties = new Properties()

View File

@@ -18,6 +18,7 @@
package org.apache.spark.sql.hudi.command package org.apache.spark.sql.hudi.command
import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _} import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.hive.ddl.HiveSyncMode
@@ -58,7 +59,12 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
val targetTable = sparkSession.sessionState.catalog val targetTable = sparkSession.sessionState.catalog
.getTableMetadata(tableId) .getTableMetadata(tableId)
val path = getTableLocation(targetTable, sparkSession) val path = getTableLocation(targetTable, sparkSession)
val conf = sparkSession.sessionState.newHadoopConf()
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(path)
.setConf(conf)
.build()
val tableConfig = metaClient.getTableConfig
val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties)
assert(primaryColumns.nonEmpty, assert(primaryColumns.nonEmpty,
@@ -66,13 +72,14 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Runnab
withSparkConf(sparkSession, targetTable.storage.properties) { withSparkConf(sparkSession, targetTable.storage.properties) {
Map( Map(
"path" -> path, "path" -> path,
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
TBL_NAME.key -> tableId.table, TBL_NAME.key -> tableId.table,
HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning,
KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName,
OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL, OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_STYLE_PARTITIONING.key -> "true",
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
) )

View File

@@ -21,12 +21,14 @@ import org.apache.avro.Schema
import org.apache.avro.generic.{GenericRecord, IndexedRecord} import org.apache.avro.generic.{GenericRecord, IndexedRecord}
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord} import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord}
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient}
import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.common.util.{Option => HOption}
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.exception.HoodieDuplicateKeyException import org.apache.hudi.exception.HoodieDuplicateKeyException
import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.MultiPartKeysValueExtractor
import org.apache.hudi.hive.ddl.HiveSyncMode import org.apache.hudi.hive.ddl.HiveSyncMode
import org.apache.hudi.keygen.ComplexKeyGenerator
import org.apache.hudi.sql.InsertMode import org.apache.hudi.sql.InsertMode
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils} import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils}
import org.apache.spark.internal.Logging import org.apache.spark.internal.Logging
@@ -90,7 +92,6 @@ object InsertIntoHoodieTableCommand extends Logging {
// for insert into or insert overwrite partition we use append mode. // for insert into or insert overwrite partition we use append mode.
SaveMode.Append SaveMode.Append
} }
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(config)
val conf = sparkSession.sessionState.conf val conf = sparkSession.sessionState.conf
val alignedQuery = alignOutputFields(query, table, insertPartitions, conf) val alignedQuery = alignOutputFields(query, table, insertPartitions, conf)
// If we create dataframe using the Dataset.ofRows(sparkSession, alignedQuery), // If we create dataframe using the Dataset.ofRows(sparkSession, alignedQuery),
@@ -100,7 +101,7 @@ object InsertIntoHoodieTableCommand extends Logging {
val inputDF = sparkSession.createDataFrame( val inputDF = sparkSession.createDataFrame(
Dataset.ofRows(sparkSession, alignedQuery).rdd, alignedQuery.schema) Dataset.ofRows(sparkSession, alignedQuery).rdd, alignedQuery.schema)
val success = val success =
HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, parameters, inputDF)._1 HoodieSparkSqlWriter.write(sparkSession.sqlContext, mode, config, inputDF)._1
if (success) { if (success) {
if (refreshTable) { if (refreshTable) {
sparkSession.catalog.refreshTable(table.identifier.unquotedString) sparkSession.catalog.refreshTable(table.identifier.unquotedString)
@@ -197,20 +198,43 @@ object InsertIntoHoodieTableCommand extends Logging {
val parameters = withSparkConf(sparkSession, options)() val parameters = withSparkConf(sparkSession, options)()
val tableType = parameters.getOrElse(TABLE_TYPE.key, TABLE_TYPE.defaultValue) val tableType = parameters.getOrElse(TABLE_TYPE.key, TABLE_TYPE.defaultValue)
val primaryColumns = HoodieOptionConfig.getPrimaryColumns(options)
val partitionFields = table.partitionColumnNames.mkString(",") val partitionFields = table.partitionColumnNames.mkString(",")
val path = getTableLocation(table, sparkSession) val path = getTableLocation(table, sparkSession)
val conf = sparkSession.sessionState.newHadoopConf()
val isTableExists = tableExistsInPath(path, conf)
val tableConfig = if (isTableExists) {
HoodieTableMetaClient.builder()
.setBasePath(path)
.setConf(conf)
.build()
.getTableConfig
} else {
null
}
val hiveStylePartitioningEnable = if (null == tableConfig || null == tableConfig.getHiveStylePartitioningEnable) {
"true"
} else {
tableConfig.getHiveStylePartitioningEnable
}
val urlEncodePartitioning = if (null == tableConfig || null == tableConfig.getUrlEncodePartitoning) {
"false"
} else {
tableConfig.getUrlEncodePartitoning
}
val keyGeneratorClassName = if (null == tableConfig || null == tableConfig.getKeyGeneratorClassName) {
if (primaryColumns.nonEmpty) {
classOf[ComplexKeyGenerator].getCanonicalName
} else {
classOf[UuidKeyGenerator].getCanonicalName
}
} else {
tableConfig.getKeyGeneratorClassName
}
val tableSchema = table.schema val tableSchema = table.schema
val primaryColumns = HoodieOptionConfig.getPrimaryColumns(options)
val keyGenClass = if (primaryColumns.nonEmpty) {
classOf[SqlKeyGenerator].getCanonicalName
} else {
classOf[UuidKeyGenerator].getName
}
val dropDuplicate = sparkSession.conf val dropDuplicate = sparkSession.conf
.getOption(INSERT_DROP_DUPS.key) .getOption(INSERT_DROP_DUPS.key)
.getOrElse(INSERT_DROP_DUPS.defaultValue) .getOrElse(INSERT_DROP_DUPS.defaultValue)
@@ -267,7 +291,9 @@ object InsertIntoHoodieTableCommand extends Logging {
TBL_NAME.key -> table.identifier.table, TBL_NAME.key -> table.identifier.table,
PRECOMBINE_FIELD.key -> tableSchema.fields.last.name, PRECOMBINE_FIELD.key -> tableSchema.fields.last.name,
OPERATION.key -> operation, OPERATION.key -> operation,
KEYGENERATOR_CLASS_NAME.key -> keyGenClass, HIVE_STYLE_PARTITIONING.key -> hiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> urlEncodePartitioning,
KEYGENERATOR_CLASS_NAME.key -> keyGeneratorClassName,
RECORDKEY_FIELD.key -> primaryColumns.mkString(","), RECORDKEY_FIELD.key -> primaryColumns.mkString(","),
PARTITIONPATH_FIELD.key -> partitionFields, PARTITIONPATH_FIELD.key -> partitionFields,
PAYLOAD_CLASS_NAME.key -> payloadClassName, PAYLOAD_CLASS_NAME.key -> payloadClassName,
@@ -279,10 +305,8 @@ object InsertIntoHoodieTableCommand extends Logging {
HIVE_DATABASE.key -> table.identifier.database.getOrElse("default"), HIVE_DATABASE.key -> table.identifier.database.getOrElse("default"),
HIVE_TABLE.key -> table.identifier.table, HIVE_TABLE.key -> table.identifier.table,
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_STYLE_PARTITIONING.key -> "true",
HIVE_PARTITION_FIELDS.key -> partitionFields, HIVE_PARTITION_FIELDS.key -> partitionFields,
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING.key -> "true",
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200",
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL SqlKeyGenerator.PARTITION_SCHEMA -> table.partitionSchema.toDDL

View File

@@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi.command
import org.apache.avro.Schema import org.apache.avro.Schema
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.MultiPartKeysValueExtractor
@@ -34,7 +35,6 @@ import org.apache.spark.sql.hudi.command.payload.ExpressionPayload
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.hudi.{HoodieOptionConfig, SerDeUtils}
import org.apache.spark.sql.types.{BooleanType, StructType} import org.apache.spark.sql.types.{BooleanType, StructType}
import java.util.Base64 import java.util.Base64
/** /**
@@ -419,7 +419,12 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
val targetTableDb = targetTableIdentify.database.getOrElse("default") val targetTableDb = targetTableIdentify.database.getOrElse("default")
val targetTableName = targetTableIdentify.identifier val targetTableName = targetTableIdentify.identifier
val path = getTableLocation(targetTable, sparkSession) val path = getTableLocation(targetTable, sparkSession)
val conf = sparkSession.sessionState.newHadoopConf()
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(path)
.setConf(conf)
.build()
val tableConfig = metaClient.getTableConfig
val options = targetTable.storage.properties val options = targetTable.storage.properties
val definedPk = HoodieOptionConfig.getPrimaryColumns(options) val definedPk = HoodieOptionConfig.getPrimaryColumns(options)
// TODO Currently the mergeEqualConditionKeys must be the same the primary key. // TODO Currently the mergeEqualConditionKeys must be the same the primary key.
@@ -429,31 +434,30 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Runnab
} }
// Enable the hive sync by default if spark have enable the hive metastore. // Enable the hive sync by default if spark have enable the hive metastore.
val enableHive = isEnableHive(sparkSession) val enableHive = isEnableHive(sparkSession)
HoodieWriterUtils.parametersWithWriteDefaults(
withSparkConf(sparkSession, options) { withSparkConf(sparkSession, options) {
Map( Map(
"path" -> path, "path" -> path,
RECORDKEY_FIELD.key -> targetKey2SourceExpression.keySet.mkString(","), RECORDKEY_FIELD.key -> targetKey2SourceExpression.keySet.mkString(","),
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field PRECOMBINE_FIELD.key -> targetKey2SourceExpression.keySet.head, // set a default preCombine field
TBL_NAME.key -> targetTableName, TBL_NAME.key -> targetTableName,
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName, PAYLOAD_CLASS_NAME.key -> classOf[ExpressionPayload].getCanonicalName,
HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning,
KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName,
META_SYNC_ENABLED.key -> enableHive.toString, META_SYNC_ENABLED.key -> enableHive.toString,
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
HIVE_USE_JDBC.key -> "false", HIVE_USE_JDBC.key -> "false",
HIVE_DATABASE.key -> targetTableDb, HIVE_DATABASE.key -> targetTableDb,
HIVE_TABLE.key -> targetTableName, HIVE_TABLE.key -> targetTableName,
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_STYLE_PARTITIONING.key -> "true",
HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING.key -> "true", // enable the url decode for sql.
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
) )
}) }
} }
} }

View File

@@ -18,11 +18,13 @@
package org.apache.spark.sql.hudi.command package org.apache.spark.sql.hudi.command
import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS} import java.util.concurrent.TimeUnit.{MICROSECONDS, MILLISECONDS}
import org.apache.avro.generic.GenericRecord import org.apache.avro.generic.GenericRecord
import org.apache.hudi.common.config.TypedProperties import org.apache.hudi.common.config.TypedProperties
import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.common.util.PartitionPathEncodeUtils
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.keygen.{BaseKeyGenerator, ComplexKeyGenerator, KeyGenUtils, SparkKeyGeneratorInterface} import org.apache.hudi.keygen._
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
import org.apache.spark.sql.Row import org.apache.spark.sql.Row
import org.apache.spark.sql.types.{StructType, TimestampType} import org.apache.spark.sql.types.{StructType, TimestampType}
import org.joda.time.format.{DateTimeFormat, DateTimeFormatter} import org.joda.time.format.{DateTimeFormat, DateTimeFormatter}
@@ -48,7 +50,8 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props)
val keyGenProps = new TypedProperties() val keyGenProps = new TypedProperties()
keyGenProps.putAll(props) keyGenProps.putAll(props)
keyGenProps.remove(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME) keyGenProps.remove(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME)
keyGenProps.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, beforeKeyGenClassName) val convertedKeyGenClassName = SqlKeyGenerator.getRealKeyGenClassName(props)
keyGenProps.put(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, convertedKeyGenClassName)
Some(KeyGenUtils.createKeyGeneratorByClassName(keyGenProps)) Some(KeyGenUtils.createKeyGeneratorByClassName(keyGenProps))
} else { } else {
None None
@@ -64,7 +67,7 @@ class SqlKeyGenerator(props: TypedProperties) extends ComplexKeyGenerator(props)
} }
override def getRecordKey(row: Row): String = { override def getRecordKey(row: Row): String = {
if (originKeyGen.isDefined && originKeyGen.get.isInstanceOf[SparkKeyGeneratorInterface]) { if (originKeyGen.isDefined) {
originKeyGen.get.asInstanceOf[SparkKeyGeneratorInterface].getRecordKey(row) originKeyGen.get.asInstanceOf[SparkKeyGeneratorInterface].getRecordKey(row)
} else { } else {
super.getRecordKey(row) super.getRecordKey(row)
@@ -121,4 +124,13 @@ object SqlKeyGenerator {
val ORIGIN_KEYGEN_CLASS_NAME = "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 timestampTimeFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss")
private val sqlTimestampFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S") private val sqlTimestampFormat = DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss.S")
def getRealKeyGenClassName(props: TypedProperties): String = {
val beforeKeyGenClassName = props.getString(SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME, null)
if (beforeKeyGenClassName != null) {
HoodieSparkKeyGeneratorFactory.convertToSparkKeyGenerator(beforeKeyGenClassName)
} else {
classOf[ComplexKeyGenerator].getCanonicalName
}
}
} }

View File

@@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi.command
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.model.HoodieRecord import org.apache.hudi.common.model.HoodieRecord
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
import org.apache.hudi.hive.MultiPartKeysValueExtractor import org.apache.hudi.hive.MultiPartKeysValueExtractor
@@ -85,7 +86,12 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
val targetTable = sparkSession.sessionState.catalog val targetTable = sparkSession.sessionState.catalog
.getTableMetadata(tableId) .getTableMetadata(tableId)
val path = getTableLocation(targetTable, sparkSession) val path = getTableLocation(targetTable, sparkSession)
val conf = sparkSession.sessionState.newHadoopConf()
val metaClient = HoodieTableMetaClient.builder()
.setBasePath(path)
.setConf(conf)
.build()
val tableConfig = metaClient.getTableConfig
val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties) val primaryColumns = HoodieOptionConfig.getPrimaryColumns(targetTable.storage.properties)
assert(primaryColumns.nonEmpty, assert(primaryColumns.nonEmpty,
@@ -95,9 +101,11 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
Map( Map(
"path" -> path, "path" -> path,
RECORDKEY_FIELD.key -> primaryColumns.mkString(","), RECORDKEY_FIELD.key -> primaryColumns.mkString(","),
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
PRECOMBINE_FIELD.key -> primaryColumns.head, //set the default preCombine field. PRECOMBINE_FIELD.key -> primaryColumns.head, //set the default preCombine field.
TBL_NAME.key -> tableId.table, TBL_NAME.key -> tableId.table,
HIVE_STYLE_PARTITIONING.key -> tableConfig.getHiveStylePartitioningEnable,
URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitoning,
KEYGENERATOR_CLASS_NAME.key -> tableConfig.getKeyGeneratorClassName,
OPERATION.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, OPERATION.key -> DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL,
PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","), PARTITIONPATH_FIELD.key -> targetTable.partitionColumnNames.mkString(","),
META_SYNC_ENABLED.key -> enableHive.toString, META_SYNC_ENABLED.key -> enableHive.toString,
@@ -107,9 +115,7 @@ case class UpdateHoodieTableCommand(updateTable: UpdateTable) extends RunnableCo
HIVE_TABLE.key -> tableId.table, HIVE_TABLE.key -> tableId.table,
HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","), HIVE_PARTITION_FIELDS.key -> targetTable.partitionColumnNames.mkString(","),
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName, HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
URL_ENCODE_PARTITIONING.key -> "true",
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
HIVE_STYLE_PARTITIONING.key -> "true",
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200", HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL SqlKeyGenerator.PARTITION_SCHEMA -> targetTable.partitionSchema.toDDL
) )

View File

@@ -30,7 +30,7 @@ import org.apache.hudi.exception.HoodieException
import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode import org.apache.hudi.execution.bulkinsert.BulkInsertSortMode
import org.apache.hudi.functional.TestBootstrap import org.apache.hudi.functional.TestBootstrap
import org.apache.hudi.hive.HiveSyncConfig import org.apache.hudi.hive.HiveSyncConfig
import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator} import org.apache.hudi.keygen.{ComplexKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
import org.apache.hudi.testutils.DataSourceTestUtils import org.apache.hudi.testutils.DataSourceTestUtils
import org.apache.spark.SparkContext import org.apache.spark.SparkContext
import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.api.java.JavaSparkContext
@@ -48,8 +48,10 @@ import org.scalatest.Matchers.{assertResult, be, convertToAnyShouldWrapper, inte
import java.time.Instant import java.time.Instant
import java.util import java.util
import java.util.{Collections, Date, UUID} import java.util.{Collections, Date, UUID}
import scala.collection.JavaConversions._ import scala.collection.JavaConversions._
import scala.collection.JavaConverters import scala.collection.JavaConverters
import scala.util.control.NonFatal
/** /**
* Test suite for SparkSqlWriter class. * Test suite for SparkSqlWriter class.
@@ -161,7 +163,6 @@ class HoodieSparkSqlWriterSuite {
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields)) .updated(HoodieTableConfig.POPULATE_META_FIELDS.key(), String.valueOf(populateMetaFields))
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name()) .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), sortMode.name())
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts // generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
@@ -175,7 +176,7 @@ class HoodieSparkSqlWriterSuite {
val recordsSeq = convertRowListToSeq(records) val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
// collect all partition paths to issue read of parquet files // collect all partition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
@@ -242,21 +243,19 @@ class HoodieSparkSqlWriterSuite {
//create a new table //create a new table
val fooTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName, val fooTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> hoodieFooTableName,
"hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4") "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val dataFrame = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) val dataFrame = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime)))
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, dataFrame) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, dataFrame)
//on same path try append with different("hoodie_bar_tbl") table name which should throw an exception //on same path try append with different("hoodie_bar_tbl") table name which should throw an exception
val barTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl", val barTableModifier = Map("path" -> tempBasePath, HoodieWriteConfig.TBL_NAME.key -> "hoodie_bar_tbl",
"hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4") "hoodie.insert.shuffle.parallelism" -> "4", "hoodie.upsert.shuffle.parallelism" -> "4")
val barTableParams = HoodieWriterUtils.parametersWithWriteDefaults(barTableModifier)
val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime))) val dataFrame2 = spark.createDataFrame(Seq(StringLongTest(UUID.randomUUID().toString, new Date().getTime)))
val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableParams, dataFrame2)) val tableAlreadyExistException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, barTableModifier, dataFrame2))
assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist")) assert(tableAlreadyExistException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
//on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception //on same path try append with delete operation and different("hoodie_bar_tbl") table name which should throw an exception
val deleteTableParams = barTableParams ++ Map(OPERATION.key -> "delete") val deleteTableModifier = barTableModifier ++ Map(OPERATION.key -> "delete")
val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableParams, dataFrame2)) val deleteCmdException = intercept[HoodieException](HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, deleteTableModifier, dataFrame2))
assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist")) assert(deleteCmdException.getMessage.contains("hoodie table with name " + hoodieFooTableName + " already exist"))
} }
@@ -295,7 +294,6 @@ class HoodieSparkSqlWriterSuite {
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name()) .updated(HoodieWriteConfig.BULK_INSERT_SORT_MODE.key(), BulkInsertSortMode.NONE.name())
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts // generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
@@ -304,7 +302,7 @@ class HoodieSparkSqlWriterSuite {
val df = spark.createDataFrame(sc.parallelize(inserts), structType) val df = spark.createDataFrame(sc.parallelize(inserts), structType)
try { try {
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
Assertions.fail("Should have thrown exception") Assertions.fail("Should have thrown exception")
} catch { } catch {
case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back")) case e: HoodieException => assertTrue(e.getMessage.contains("hoodie.populate.meta.fields already disabled for the table. Can't be re-enabled back"))
@@ -323,7 +321,6 @@ class HoodieSparkSqlWriterSuite {
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
.updated(INSERT_DROP_DUPS.key, "true") .updated(INSERT_DROP_DUPS.key, "true")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts // generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
@@ -332,7 +329,7 @@ class HoodieSparkSqlWriterSuite {
val recordsSeq = convertRowListToSeq(records) val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(spark.sparkContext.parallelize(recordsSeq), structType) val df = spark.createDataFrame(spark.sparkContext.parallelize(recordsSeq), structType)
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
fail("Drop duplicates with bulk insert in row writing should have thrown exception") fail("Drop duplicates with bulk insert in row writing should have thrown exception")
} catch { } catch {
case e: HoodieException => assertTrue(e.getMessage.contains("Dropping duplicates with bulk_insert in row writer path is not supported yet")) case e: HoodieException => assertTrue(e.getMessage.contains("Dropping duplicates with bulk_insert in row writer path is not supported yet"))
@@ -348,7 +345,6 @@ class HoodieSparkSqlWriterSuite {
//create a new table //create a new table
val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) val fooTableModifier = commonTableModifier.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false") .updated(DataSourceWriteOptions.INSERT_DROP_DUPS.key, "false")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts // generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
@@ -357,7 +353,7 @@ class HoodieSparkSqlWriterSuite {
val recordsSeq = convertRowListToSeq(records) val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier - DataSourceWriteOptions.PRECOMBINE_FIELD.key, df)
// collect all partition paths to issue read of parquet files // collect all partition paths to issue read of parquet files
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
@@ -384,7 +380,6 @@ class HoodieSparkSqlWriterSuite {
val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4") val fooTableModifier = commonTableModifier.updated("hoodie.bulkinsert.shuffle.parallelism", "4")
.updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL) .updated(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
.updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true") .updated(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, "true")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH, val partitions = Seq(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH,
HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH) HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH)
val fullPartitionPaths = new Array[String](3) val fullPartitionPaths = new Array[String](3)
@@ -400,7 +395,7 @@ class HoodieSparkSqlWriterSuite {
val recordsSeq = convertRowListToSeq(records) val recordsSeq = convertRowListToSeq(records)
val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df)
// Fetch records from entire dataset // Fetch records from entire dataset
val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2)) val actualDf = sqlContext.read.parquet(fullPartitionPaths(0), fullPartitionPaths(1), fullPartitionPaths(2))
// remove metadata columns so that expected and actual DFs can be compared as is // remove metadata columns so that expected and actual DFs can be compared as is
@@ -450,7 +445,7 @@ class HoodieSparkSqlWriterSuite {
new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName, new JavaSparkContext(sc), modifiedSchema.toString, tempBasePath, hoodieFooTableName,
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df, Option.empty, Option(client)) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df, Option.empty, Option(client))
// Verify that asynchronous compaction is not scheduled // Verify that asynchronous compaction is not scheduled
verify(client, times(0)).scheduleCompaction(any()) verify(client, times(0)).scheduleCompaction(any())
// Verify that HoodieWriteClient is closed correctly // Verify that HoodieWriteClient is closed correctly
@@ -511,7 +506,7 @@ class HoodieSparkSqlWriterSuite {
hoodieFooTableName, hoodieFooTableName,
mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]])
HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableParams, spark.emptyDataFrame, Option.empty, HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableModifier, spark.emptyDataFrame, Option.empty,
Option(client)) Option(client))
// Verify that HoodieWriteClient is closed correctly // Verify that HoodieWriteClient is closed correctly
@@ -556,7 +551,6 @@ class HoodieSparkSqlWriterSuite {
//create a new table //create a new table
val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, tableType) val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, tableType)
.updated(DataSourceWriteOptions.RECONCILE_SCHEMA.key, "true") .updated(DataSourceWriteOptions.RECONCILE_SCHEMA.key, "true")
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
// generate the inserts // generate the inserts
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
@@ -564,7 +558,7 @@ class HoodieSparkSqlWriterSuite {
var records = DataSourceTestUtils.generateRandomRows(10) var records = DataSourceTestUtils.generateRandomRows(10)
var recordsSeq = convertRowListToSeq(records) var recordsSeq = convertRowListToSeq(records)
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableModifier, df1)
val snapshotDF1 = spark.read.format("org.apache.hudi") val snapshotDF1 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
@@ -577,7 +571,7 @@ class HoodieSparkSqlWriterSuite {
// issue updates so that log files are created for MOR table // issue updates so that log files are created for MOR table
val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5)) val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType) val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, updatesDf)
val snapshotDF2 = spark.read.format("org.apache.hudi") val snapshotDF2 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
@@ -595,7 +589,7 @@ class HoodieSparkSqlWriterSuite {
recordsSeq = convertRowListToSeq(records) recordsSeq = convertRowListToSeq(records)
val df3 = spark.createDataFrame(sc.parallelize(recordsSeq), evolStructType) val df3 = spark.createDataFrame(sc.parallelize(recordsSeq), evolStructType)
// write to Hudi with new column // write to Hudi with new column
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df3) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df3)
val snapshotDF3 = spark.read.format("org.apache.hudi") val snapshotDF3 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
@@ -610,7 +604,7 @@ class HoodieSparkSqlWriterSuite {
records = DataSourceTestUtils.generateRandomRows(10) records = DataSourceTestUtils.generateRandomRows(10)
recordsSeq = convertRowListToSeq(records) recordsSeq = convertRowListToSeq(records)
val df4 = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df4 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, df4) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, df4)
val snapshotDF4 = spark.read.format("org.apache.hudi") val snapshotDF4 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
@@ -743,14 +737,13 @@ class HoodieSparkSqlWriterSuite {
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testDeletePartitionsV2(usePartitionsToDeleteConfig: Boolean): Unit = { def testDeletePartitionsV2(usePartitionsToDeleteConfig: Boolean): Unit = {
val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, HoodieTableType.COPY_ON_WRITE.name()) val fooTableModifier = getCommonParams(tempPath, hoodieFooTableName, HoodieTableType.COPY_ON_WRITE.name())
val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier)
val schema = DataSourceTestUtils.getStructTypeExampleSchema val schema = DataSourceTestUtils.getStructTypeExampleSchema
val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema) val structType = AvroConversionUtils.convertAvroSchemaToStructType(schema)
val records = DataSourceTestUtils.generateRandomRows(10) val records = DataSourceTestUtils.generateRandomRows(10)
val recordsSeq = convertRowListToSeq(records) val recordsSeq = convertRowListToSeq(records)
val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType) val df1 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
// write to Hudi // write to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableParams, df1) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Overwrite, fooTableModifier, df1)
val snapshotDF1 = spark.read.format("org.apache.hudi") val snapshotDF1 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
assertEquals(10, snapshotDF1.count()) assertEquals(10, snapshotDF1.count())
@@ -761,7 +754,7 @@ class HoodieSparkSqlWriterSuite {
val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5)) val updatesSeq = convertRowListToSeq(DataSourceTestUtils.generateUpdates(records, 5))
val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType) val updatesDf = spark.createDataFrame(sc.parallelize(updatesSeq), structType)
// write updates to Hudi // write updates to Hudi
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableParams, updatesDf) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, fooTableModifier, updatesDf)
val snapshotDF2 = spark.read.format("org.apache.hudi") val snapshotDF2 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
assertEquals(10, snapshotDF2.count()) assertEquals(10, snapshotDF2.count())
@@ -770,7 +763,7 @@ class HoodieSparkSqlWriterSuite {
// ensure 2nd batch of updates matches. // ensure 2nd batch of updates matches.
assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0) assert(updatesDf.intersect(trimmedDf2).except(updatesDf).count() == 0)
if (usePartitionsToDeleteConfig) { if (usePartitionsToDeleteConfig) {
fooTableParams.updated(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) fooTableModifier.updated(DataSourceWriteOptions.PARTITIONS_TO_DELETE.key(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)
} }
// delete partitions contains the primary key // delete partitions contains the primary key
val recordsToDelete = df1.filter(entry => { val recordsToDelete = df1.filter(entry => {
@@ -778,7 +771,7 @@ class HoodieSparkSqlWriterSuite {
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) || partitionPath.equals(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) ||
partitionPath.equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH) partitionPath.equals(HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH)
}) })
val updatedParams = fooTableParams.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name()) val updatedParams = fooTableModifier.updated(DataSourceWriteOptions.OPERATION.key(), WriteOperationType.DELETE_PARTITION.name())
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, updatedParams, recordsToDelete) HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, updatedParams, recordsToDelete)
val snapshotDF3 = spark.read.format("org.apache.hudi") val snapshotDF3 = spark.read.format("org.apache.hudi")
.load(tempBasePath + "/*/*/*/*") .load(tempBasePath + "/*/*/*/*")
@@ -819,4 +812,88 @@ class HoodieSparkSqlWriterSuite {
assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10) assert(spark.read.format("hudi").load(tempBasePath).where("age >= 2000").count() == 10)
} }
} }
/**
* Test case for no need to specify hiveStylePartitioning/urlEncodePartitioning/KeyGenerator included in HoodieTableConfig except the first time write
*/
@Test
def testToWriteWithoutParametersIncludedInHoodieTableConfig(): Unit = {
val _spark = spark
import _spark.implicits._
val df = Seq((1, "a1", 10, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
val options = Map(
DataSourceWriteOptions.RECORDKEY_FIELD.key -> "id",
DataSourceWriteOptions.PRECOMBINE_FIELD.key -> "ts",
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "dt"
)
// case 1: test table which created by sql
val (tableName1, tablePath1) = ("hoodie_test_params_1", s"$tempBasePath" + "_1")
spark.sql(
s"""
| create table $tableName1 (
| id int,
| name string,
| price double,
| ts long,
| dt string
| ) using hudi
| partitioned by (dt)
| options (
| primaryKey = 'id'
| )
| location '$tablePath1'
""".stripMargin)
val tableConfig1 = HoodieTableMetaClient.builder()
.setConf(spark.sparkContext.hadoopConfiguration)
.setBasePath(tablePath1).build().getTableConfig
assert(tableConfig1.getHiveStylePartitioningEnable == "true")
assert(tableConfig1.getUrlEncodePartitoning == "false")
assert(tableConfig1.getKeyGeneratorClassName == classOf[ComplexKeyGenerator].getName)
df.write.format("hudi")
.options(options)
.option(HoodieWriteConfig.TBL_NAME.key, tableName1)
.mode(SaveMode.Append).save(tablePath1)
assert(spark.read.format("hudi").load(tablePath1 + "/*").count() == 1)
// case 2: test table which created by dataframe
val (tableName2, tablePath2) = ("hoodie_test_params_2", s"$tempBasePath" + "_2")
// the first write need to specify params
df.write.format("hudi")
.options(options)
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
.option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, "true")
.option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[SimpleKeyGenerator].getName)
.mode(SaveMode.Overwrite).save(tablePath2)
val tableConfig2 = HoodieTableMetaClient.builder()
.setConf(spark.sparkContext.hadoopConfiguration)
.setBasePath(tablePath2).build().getTableConfig
assert(tableConfig2.getHiveStylePartitioningEnable == "false")
assert(tableConfig2.getUrlEncodePartitoning == "true")
assert(tableConfig2.getKeyGeneratorClassName == classOf[SimpleKeyGenerator].getName)
val df2 = Seq((2, "a2", 20, 1000, "2021-10-16")).toDF("id", "name", "value", "ts", "dt")
// raise exception when use params which is not same with HoodieTableConfig
try {
df2.write.format("hudi")
.options(options)
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
.option(HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key, classOf[ComplexKeyGenerator].getName)
.mode(SaveMode.Append).save(tablePath2)
} catch {
case NonFatal(e) =>
assert(e.getMessage.contains("Config conflict"))
assert(e.getMessage.contains(
s"${HoodieWriteConfig.KEYGENERATOR_CLASS_NAME.key}\t${classOf[ComplexKeyGenerator].getName}\t${classOf[SimpleKeyGenerator].getName}"))
}
// do not need to specify hiveStylePartitioning/urlEncodePartitioning/KeyGenerator params
df2.write.format("hudi")
.options(options)
.option(HoodieWriteConfig.TBL_NAME.key, tableName2)
.mode(SaveMode.Append).save(tablePath2)
val data = spark.read.format("hudi").load(tablePath2 + "/*")
assert(data.count() == 2)
assert(data.select("_hoodie_partition_path").map(_.getString(0)).distinct.collect.head == "dt=2021-10-16")
}
} }

View File

@@ -17,6 +17,8 @@
package org.apache.hudi package org.apache.hudi
import java.util.Properties
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.HoodieTableMetaClient
@@ -58,6 +60,7 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
) )
@BeforeEach override def setUp() { @BeforeEach override def setUp() {
setTableName("hoodie_test")
initPath() initPath()
initSparkContexts() initSparkContexts()
spark = sqlContext.sparkSession spark = sqlContext.sparkSession
@@ -71,6 +74,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testPartitionSchema(partitionEncode: Boolean): Unit = { def testPartitionSchema(partitionEncode: Boolean): Unit = {
val props = new Properties()
props.setProperty(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, String.valueOf(partitionEncode))
initMetaClient(props)
val records1 = dataGen.generateInsertsContainsAllPartitions("000", 100) val records1 = dataGen.generateInsertsContainsAllPartitions("000", 100)
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2)) val inputDF1 = spark.read.json(spark.sparkContext.parallelize(recordsToStrings(records1), 2))
inputDF1.write.format("hudi") inputDF1.write.format("hudi")
@@ -128,6 +134,9 @@ class TestHoodieFileIndex extends HoodieClientTestBase {
@ParameterizedTest @ParameterizedTest
@ValueSource(booleans = Array(true, false)) @ValueSource(booleans = Array(true, false))
def testPartitionPruneWithPartitionEncode(partitionEncode: Boolean): Unit = { def testPartitionPruneWithPartitionEncode(partitionEncode: Boolean): Unit = {
val props = new Properties()
props.setProperty(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key, String.valueOf(partitionEncode))
initMetaClient(props)
val partitions = Array("2021/03/08", "2021/03/09", "2021/03/10", "2021/03/11", "2021/03/12") val partitions = Array("2021/03/08", "2021/03/09", "2021/03/10", "2021/03/11", "2021/03/12")
val newDataGen = new HoodieTestDataGenerator(partitions) val newDataGen = new HoodieTestDataGenerator(partitions)
val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100) val records1 = newDataGen.generateInsertsContainsAllPartitions("000", 100)

View File

@@ -154,7 +154,9 @@ class TestDataSourceForBootstrap {
// Perform bootstrap // Perform bootstrap
val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit( val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, Some("datestr")) DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
Some("datestr"),
Map(DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true"))
// Read bootstrapped table and verify count // Read bootstrapped table and verify count
val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*") val hoodieROViewDF1 = spark.read.format("hudi").load(basePath + "/*")
@@ -473,11 +475,13 @@ class TestDataSourceForBootstrap {
} }
def runMetadataBootstrapAndVerifyCommit(tableType: String, def runMetadataBootstrapAndVerifyCommit(tableType: String,
partitionColumns: Option[String] = None): String = { partitionColumns: Option[String] = None,
extraOpts: Map[String, String] = Map.empty): String = {
val bootstrapDF = spark.emptyDataFrame val bootstrapDF = spark.emptyDataFrame
bootstrapDF.write bootstrapDF.write
.format("hudi") .format("hudi")
.options(commonOpts) .options(commonOpts)
.options(extraOpts)
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL)
.option(DataSourceWriteOptions.TABLE_TYPE.key, tableType) .option(DataSourceWriteOptions.TABLE_TYPE.key, tableType)
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, partitionColumns.getOrElse("")) .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, partitionColumns.getOrElse(""))

View File

@@ -17,6 +17,8 @@
package org.apache.hudi.functional package org.apache.hudi.functional
import java.util.Properties
import org.apache.hadoop.fs.Path import org.apache.hadoop.fs.Path
import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.DataSourceWriteOptions._
import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.config.HoodieMetadataConfig
@@ -61,6 +63,7 @@ class TestMORDataSource extends HoodieClientTestBase {
val updatedVerificationVal: String = "driver_update" val updatedVerificationVal: String = "driver_update"
@BeforeEach override def setUp() { @BeforeEach override def setUp() {
setTableName("hoodie_test")
initPath() initPath()
initSparkContexts() initSparkContexts()
spark = sqlContext.sparkSession spark = sqlContext.sparkSession

View File

@@ -46,6 +46,7 @@ class TestTimeTravelQuery extends HoodieClientTestBase {
) )
@BeforeEach override def setUp() { @BeforeEach override def setUp() {
setTableName("hoodie_test")
initPath() initPath()
initSparkContexts() initSparkContexts()
spark = sqlContext.sparkSession spark = sqlContext.sparkSession