[HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175)
- Refactor hive sync tool / config to use reflection and standardize configs Co-authored-by: sivabalan <n.siva.b@gmail.com> Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local> Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -21,6 +21,8 @@ import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.spark.sql.SaveMode;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.HoodieDataSourceHelpers;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
@@ -43,14 +45,14 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
|
||||
option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr").
|
||||
option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts").
|
||||
option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor").
|
||||
option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor").
|
||||
option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default").
|
||||
option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
|
||||
option(DataSourceWriteOptions.HIVE_USER.key(), "hive").
|
||||
option(DataSourceWriteOptions.HIVE_PASS.key(), "hive").
|
||||
option(DataSourceWriteOptions.HIVE_SYNC_ENABLED.key(), "true").
|
||||
option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key(), "datestr").
|
||||
option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
|
||||
option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor").
|
||||
option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default").
|
||||
option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
|
||||
option(HiveSyncConfig.HIVE_USER.key(), "hive").
|
||||
option(HiveSyncConfig.HIVE_PASS.key(), "hive").
|
||||
option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true").
|
||||
option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr").
|
||||
option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
|
||||
option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true").
|
||||
mode(SaveMode.Overwrite).
|
||||
save("/user/hive/warehouse/stock_ticks_derived_mor");
|
||||
@@ -75,14 +77,14 @@ spark.sql("select key, `_hoodie_partition_path` as datestr, symbol, ts, open, cl
|
||||
option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key(), "datestr").
|
||||
option(DataSourceWriteOptions.PRECOMBINE_FIELD.key(), "ts").
|
||||
option(HoodieWriteConfig.TBL_NAME.key(), "stock_ticks_derived_mor_bs").
|
||||
option(DataSourceWriteOptions.HIVE_TABLE.key(), "stock_ticks_derived_mor_bs").
|
||||
option(DataSourceWriteOptions.HIVE_DATABASE.key(), "default").
|
||||
option(DataSourceWriteOptions.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
|
||||
option(DataSourceWriteOptions.HIVE_USER.key(), "hive").
|
||||
option(DataSourceWriteOptions.HIVE_PASS.key(), "hive").
|
||||
option(DataSourceWriteOptions.HIVE_SYNC_ENABLED.key(), "true").
|
||||
option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key(), "datestr").
|
||||
option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
|
||||
option(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "stock_ticks_derived_mor_bs").
|
||||
option(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "default").
|
||||
option(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://hiveserver:10000").
|
||||
option(HiveSyncConfig.HIVE_USER.key(), "hive").
|
||||
option(HiveSyncConfig.HIVE_PASS.key(), "hive").
|
||||
option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true").
|
||||
option(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr").
|
||||
option(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), classOf[MultiPartKeysValueExtractor].getCanonicalName).
|
||||
option(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true").
|
||||
mode(SaveMode.Overwrite).
|
||||
save("/user/hive/warehouse/stock_ticks_derived_mor_bs");
|
||||
|
||||
@@ -105,7 +105,7 @@ public class HoodieWriteConfig extends HoodieConfig {
|
||||
public static final String DELTASTREAMER_CHECKPOINT_KEY = "deltastreamer.checkpoint.key";
|
||||
|
||||
public static final ConfigProperty<String> TBL_NAME = ConfigProperty
|
||||
.key("hoodie.table.name")
|
||||
.key(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Table name that will be used for registering with metastores like HMS. Needs to be same across runs.");
|
||||
|
||||
|
||||
@@ -82,6 +82,8 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
public static final String HOODIE_PROPERTIES_FILE_BACKUP = "hoodie.properties.backup";
|
||||
public static final String HOODIE_WRITE_TABLE_NAME_KEY = "hoodie.datasource.write.table.name";
|
||||
public static final String HOODIE_TABLE_NAME_KEY = "hoodie.table.name";
|
||||
|
||||
public static final ConfigProperty<String> DATABASE_NAME = ConfigProperty
|
||||
.key("hoodie.database.name")
|
||||
@@ -90,7 +92,7 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
+ "we can set it to limit the table name under a specific database");
|
||||
|
||||
public static final ConfigProperty<String> NAME = ConfigProperty
|
||||
.key("hoodie.table.name")
|
||||
.key(HOODIE_TABLE_NAME_KEY)
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs.");
|
||||
|
||||
|
||||
@@ -92,6 +92,24 @@ public class ReflectionUtils {
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the clazz has the target constructor or not.
|
||||
*
|
||||
* When catch {@link HoodieException} from {@link #loadClass}, it's inconvenient to say if the exception was thrown
|
||||
* due to the instantiation's own logic or missing constructor.
|
||||
*
|
||||
* TODO: ReflectionUtils should throw a specific exception to indicate Reflection problem.
|
||||
*/
|
||||
public static boolean hasConstructor(String clazz, Class<?>[] constructorArgTypes) {
|
||||
try {
|
||||
getClass(clazz).getConstructor(constructorArgTypes);
|
||||
return true;
|
||||
} catch (NoSuchMethodException e) {
|
||||
LOG.warn("Unable to instantiate class " + clazz, e);
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Creates an instance of the given class. Constructor arg types are inferred.
|
||||
*/
|
||||
|
||||
@@ -18,17 +18,19 @@
|
||||
|
||||
package org.apache.hudi.integ.testsuite.dag.nodes;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig;
|
||||
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
|
||||
import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
|
||||
import java.sql.Connection;
|
||||
import java.sql.DriverManager;
|
||||
import java.sql.ResultSet;
|
||||
import java.sql.SQLException;
|
||||
import java.sql.Statement;
|
||||
|
||||
/**
|
||||
* A hive query node in the DAG of operations for a workflow. used to perform a hive query with given config.
|
||||
@@ -46,13 +48,14 @@ public class HiveQueryNode extends DagNode<Boolean> {
|
||||
public void execute(ExecutionContext executionContext, int curItrCount) throws Exception {
|
||||
log.info("Executing hive query node {}", this.getName());
|
||||
this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
|
||||
HiveSyncConfig hiveSyncConfig = DataSourceUtils
|
||||
.buildHiveSyncConfig(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getProps(),
|
||||
executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getCfg().targetBasePath,
|
||||
executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat);
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getProps());
|
||||
properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getCfg().targetBasePath);
|
||||
properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper()
|
||||
.getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat);
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig(properties);
|
||||
this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
|
||||
Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser,
|
||||
hiveSyncConfig.hivePass);
|
||||
|
||||
@@ -18,31 +18,30 @@
|
||||
|
||||
package org.apache.hudi.integ.testsuite.dag.nodes;
|
||||
|
||||
import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
|
||||
import org.apache.hudi.integ.testsuite.dag.ExecutionContext;
|
||||
import org.apache.hudi.sync.common.util.SyncUtilHelpers;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
/**
|
||||
* Represents a hive sync node in the DAG of operations for a workflow. Helps to sync hoodie data to hive table.
|
||||
*/
|
||||
public class HiveSyncNode extends DagNode<Boolean> {
|
||||
|
||||
private HiveServiceProvider hiveServiceProvider;
|
||||
|
||||
public HiveSyncNode(Config config) {
|
||||
this.config = config;
|
||||
this.hiveServiceProvider = new HiveServiceProvider(config);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void execute(ExecutionContext executionContext, int curItrCount) throws Exception {
|
||||
log.info("Executing hive sync node");
|
||||
this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration());
|
||||
this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter());
|
||||
this.hiveServiceProvider.stopLocalHiveServiceIfNeeded();
|
||||
}
|
||||
|
||||
public HiveServiceProvider getHiveServiceProvider() {
|
||||
return hiveServiceProvider;
|
||||
SyncUtilHelpers.runHoodieMetaSync(HiveSyncTool.class.getName(), new TypedProperties(executionContext.getHoodieTestSuiteWriter().getProps()),
|
||||
executionContext.getHoodieTestSuiteWriter().getConfiguration(),
|
||||
new Path(executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath).getFileSystem(executionContext.getHoodieTestSuiteWriter().getConfiguration()),
|
||||
executionContext.getHoodieTestSuiteWriter().getCfg().targetBasePath, executionContext.getHoodieTestSuiteWriter().getCfg().baseFileFormat);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -21,6 +21,9 @@ package org.apache.hudi.integ.testsuite.helpers;
|
||||
import java.io.IOException;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hive.service.server.HiveServer2;
|
||||
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.hive.testutils.HiveTestService;
|
||||
import org.apache.hudi.integ.testsuite.HoodieTestSuiteWriter;
|
||||
import org.apache.hudi.integ.testsuite.configuration.DeltaConfig.Config;
|
||||
@@ -46,12 +49,17 @@ public class HiveServiceProvider {
|
||||
}
|
||||
|
||||
public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) {
|
||||
HiveSyncTool hiveSyncTool;
|
||||
if (this.config.isHiveLocal()) {
|
||||
writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync()
|
||||
.syncHive(getLocalHiveServer().getHiveConf());
|
||||
hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(),
|
||||
getLocalHiveServer().getHiveConf(),
|
||||
FSUtils.getFs(writer.getWriteConfig().getBasePath(), getLocalHiveServer().getHiveConf()));
|
||||
} else {
|
||||
writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive();
|
||||
hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(),
|
||||
getLocalHiveServer().getHiveConf(),
|
||||
FSUtils.getFs(writer.getWriteConfig().getBasePath(), writer.getConfiguration()));
|
||||
}
|
||||
hiveSyncTool.syncHoodieTable();
|
||||
}
|
||||
|
||||
public void stopLocalHiveServiceIfNeeded() throws IOException {
|
||||
|
||||
@@ -22,6 +22,7 @@ import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob;
|
||||
import org.apache.hudi.integ.testsuite.HoodieTestSuiteJob.HoodieTestSuiteConfig;
|
||||
@@ -173,10 +174,10 @@ public class TestHoodieTestSuiteJob extends UtilitiesTestBase {
|
||||
// Make path selection test suite specific
|
||||
props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName());
|
||||
// Hive Configs
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "table1");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
|
||||
props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "table1");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr");
|
||||
props.setProperty(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName());
|
||||
|
||||
props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider");
|
||||
|
||||
@@ -32,8 +32,6 @@ import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.connect.ControlMessage;
|
||||
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.CustomAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.CustomKeyGenerator;
|
||||
@@ -59,7 +57,6 @@ import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -269,32 +266,4 @@ public class KafkaConnectUtils {
|
||||
ControlMessage.ConnectWriteStatus connectWriteStatus = participantInfo.getWriteStatus();
|
||||
return SerializationUtils.deserialize(connectWriteStatus.getSerializedWriteStatus().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hive Sync Config
|
||||
* Note: This method is a temporary solution.
|
||||
* Future solutions can be referred to: https://issues.apache.org/jira/browse/HUDI-3199
|
||||
*/
|
||||
public static HiveSyncConfig buildSyncConfig(TypedProperties props, String tableBasePath) {
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.basePath = tableBasePath;
|
||||
hiveSyncConfig.usePreApacheInputFormat = props.getBoolean(KafkaConnectConfigs.HIVE_USE_PRE_APACHE_INPUT_FORMAT, false);
|
||||
hiveSyncConfig.databaseName = props.getString(KafkaConnectConfigs.HIVE_DATABASE, "default");
|
||||
hiveSyncConfig.tableName = props.getString(KafkaConnectConfigs.HIVE_TABLE, "");
|
||||
hiveSyncConfig.hiveUser = props.getString(KafkaConnectConfigs.HIVE_USER, "");
|
||||
hiveSyncConfig.hivePass = props.getString(KafkaConnectConfigs.HIVE_PASS, "");
|
||||
hiveSyncConfig.jdbcUrl = props.getString(KafkaConnectConfigs.HIVE_URL, "");
|
||||
hiveSyncConfig.partitionFields = props.getStringList(KafkaConnectConfigs.HIVE_PARTITION_FIELDS, ",", Collections.emptyList());
|
||||
hiveSyncConfig.partitionValueExtractorClass =
|
||||
props.getString(KafkaConnectConfigs.HIVE_PARTITION_EXTRACTOR_CLASS, SlashEncodedDayPartitionValueExtractor.class.getName());
|
||||
hiveSyncConfig.useJdbc = props.getBoolean(KafkaConnectConfigs.HIVE_USE_JDBC, true);
|
||||
if (props.containsKey(KafkaConnectConfigs.HIVE_SYNC_MODE)) {
|
||||
hiveSyncConfig.syncMode = props.getString(KafkaConnectConfigs.HIVE_SYNC_MODE);
|
||||
}
|
||||
hiveSyncConfig.autoCreateDatabase = props.getBoolean(KafkaConnectConfigs.HIVE_AUTO_CREATE_DATABASE, true);
|
||||
hiveSyncConfig.ignoreExceptions = props.getBoolean(KafkaConnectConfigs.HIVE_IGNORE_EXCEPTIONS, false);
|
||||
hiveSyncConfig.skipROSuffix = props.getBoolean(KafkaConnectConfigs.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE, false);
|
||||
hiveSyncConfig.supportTimestamp = props.getBoolean(KafkaConnectConfigs.HIVE_SUPPORT_TIMESTAMP_TYPE, false);
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -30,22 +30,17 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.connect.transaction.TransactionCoordinator;
|
||||
import org.apache.hudi.connect.utils.KafkaConnectUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.sync.common.util.SyncUtilHelpers;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -54,7 +49,6 @@ import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
@@ -167,43 +161,10 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
|
||||
if (connectConfigs.isMetaSyncEnabled()) {
|
||||
Set<String> syncClientToolClasses = new HashSet<>(
|
||||
Arrays.asList(connectConfigs.getMetaSyncClasses().split(",")));
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
|
||||
for (String impl : syncClientToolClasses) {
|
||||
impl = impl.trim();
|
||||
switch (impl) {
|
||||
case "org.apache.hudi.hive.HiveSyncTool":
|
||||
syncHive();
|
||||
break;
|
||||
default:
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(connectConfigs.getProps());
|
||||
properties.put("basePath", tableBasePath);
|
||||
AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[] {Properties.class, FileSystem.class}, properties, fs);
|
||||
syncTool.syncHoodieTable();
|
||||
}
|
||||
SyncUtilHelpers.runHoodieMetaSync(impl.trim(), connectConfigs.getProps(), hadoopConf, fs, tableBasePath, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void syncHive() {
|
||||
HiveSyncConfig hiveSyncConfig = KafkaConnectUtils.buildSyncConfig(new TypedProperties(connectConfigs.getProps()), tableBasePath);
|
||||
String url;
|
||||
if (!StringUtils.isNullOrEmpty(hiveSyncConfig.syncMode) && HiveSyncMode.of(hiveSyncConfig.syncMode) == HiveSyncMode.HMS) {
|
||||
url = hadoopConf.get(KafkaConnectConfigs.HIVE_METASTORE_URIS);
|
||||
} else {
|
||||
url = hiveSyncConfig.jdbcUrl;
|
||||
}
|
||||
|
||||
LOG.info("Syncing target hoodie table with hive table("
|
||||
+ hiveSyncConfig.tableName
|
||||
+ "). Hive URL :"
|
||||
+ url
|
||||
+ ", basePath :" + tableBasePath);
|
||||
LOG.info("Hive Sync Conf => " + hiveSyncConfig);
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, hadoopConf);
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
hiveConf.addResource(fs.getConf());
|
||||
LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString());
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -42,6 +42,7 @@ import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.table.BulkInsertPartitioner;
|
||||
import org.apache.hudi.util.DataTypeUtils;
|
||||
|
||||
@@ -270,6 +271,11 @@ public class DataSourceUtils {
|
||||
return dropDuplicates(jssc, incomingHoodieRecords, writeConfig);
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link HiveSyncConfig} constructor directly and provide the props,
|
||||
* and set {@link HoodieSyncConfig#META_SYNC_BASE_PATH} and {@link HoodieSyncConfig#META_SYNC_BASE_FILE_FORMAT} instead.
|
||||
*/
|
||||
@Deprecated
|
||||
public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String basePath, String baseFileFormat) {
|
||||
checkRequiredProperties(props, Collections.singletonList(DataSourceWriteOptions.HIVE_TABLE().key()));
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
@@ -310,7 +316,7 @@ public class DataSourceUtils {
|
||||
hiveSyncConfig.isConditionalSync = Boolean.valueOf(props.getString(DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().key(),
|
||||
DataSourceWriteOptions.HIVE_CONDITIONAL_SYNC().defaultValue()));
|
||||
hiveSyncConfig.bucketSpec = props.getBoolean(DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().key(),
|
||||
(boolean) DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue())
|
||||
DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue())
|
||||
? HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
|
||||
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())) : null;
|
||||
if (props.containsKey(HiveExternalCatalog.CREATED_SPARK_VERSION())) {
|
||||
|
||||
@@ -25,9 +25,10 @@ import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.common.util.Option
|
||||
import org.apache.hudi.config.{HoodieClusteringConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.hive.util.ConfigUtils
|
||||
import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor}
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor}
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.sql.execution.datasources.{DataSourceUtils => SparkDataSourceUtils}
|
||||
|
||||
@@ -246,7 +247,7 @@ object DataSourceWriteOptions {
|
||||
}
|
||||
|
||||
val TABLE_NAME: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.write.table.name")
|
||||
.key(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.")
|
||||
|
||||
@@ -380,185 +381,79 @@ object DataSourceWriteOptions {
|
||||
// HIVE SYNC SPECIFIC CONFIGS
|
||||
// NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
|
||||
// unexpected issues with config getting reset
|
||||
val HIVE_SYNC_ENABLED: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, register/sync the table to Apache Hive metastore")
|
||||
|
||||
val META_SYNC_ENABLED: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.meta.sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("")
|
||||
|
||||
val HIVE_DATABASE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.database")
|
||||
.defaultValue("default")
|
||||
.withDocumentation("database to sync to")
|
||||
|
||||
val hiveTableOptKeyInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
|
||||
if (p.contains(TABLE_NAME)) {
|
||||
Option.of(p.getString(TABLE_NAME))
|
||||
} else if (p.contains(HoodieWriteConfig.TBL_NAME)) {
|
||||
Option.of(p.getString(HoodieWriteConfig.TBL_NAME))
|
||||
} else {
|
||||
Option.empty[String]()
|
||||
}
|
||||
})
|
||||
val HIVE_TABLE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.table")
|
||||
.defaultValue("unknown")
|
||||
.withInferFunction(hiveTableOptKeyInferFunc)
|
||||
.withDocumentation("table to sync to")
|
||||
|
||||
val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.base_file_format")
|
||||
.defaultValue("PARQUET")
|
||||
.withDocumentation("Base file format for the sync.")
|
||||
|
||||
val HIVE_USER: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.username")
|
||||
.defaultValue("hive")
|
||||
.withDocumentation("hive user name to use")
|
||||
|
||||
val HIVE_PASS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.password")
|
||||
.defaultValue("hive")
|
||||
.withDocumentation("hive password to use")
|
||||
|
||||
val HIVE_URL: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.jdbcurl")
|
||||
.defaultValue("jdbc:hive2://localhost:10000")
|
||||
.withDocumentation("Hive jdbc url")
|
||||
|
||||
val METASTORE_URIS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.metastore.uris")
|
||||
.defaultValue("thrift://localhost:9083")
|
||||
.withDocumentation("Hive metastore url")
|
||||
|
||||
val hivePartitionFieldsInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
|
||||
if (p.contains(PARTITIONPATH_FIELD)) {
|
||||
Option.of(p.getString(PARTITIONPATH_FIELD))
|
||||
} else {
|
||||
Option.empty[String]()
|
||||
}
|
||||
})
|
||||
val HIVE_PARTITION_FIELDS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.partition_fields")
|
||||
.defaultValue("")
|
||||
.withDocumentation("Field in the table to use for determining hive partition columns.")
|
||||
.withInferFunction(hivePartitionFieldsInferFunc)
|
||||
|
||||
val hivePartitionExtractorInferFunc = DataSourceOptionsHelper.scalaFunctionToJavaFunction((p: HoodieConfig) => {
|
||||
if (!p.contains(PARTITIONPATH_FIELD)) {
|
||||
Option.of(classOf[NonPartitionedExtractor].getName)
|
||||
} else {
|
||||
val numOfPartFields = p.getString(PARTITIONPATH_FIELD).split(",").length
|
||||
if (numOfPartFields == 1 && p.contains(HIVE_STYLE_PARTITIONING) && p.getString(HIVE_STYLE_PARTITIONING) == "true") {
|
||||
Option.of(classOf[HiveStylePartitionValueExtractor].getName)
|
||||
} else {
|
||||
Option.of(classOf[MultiPartKeysValueExtractor].getName)
|
||||
}
|
||||
}
|
||||
})
|
||||
val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.partition_extractor_class")
|
||||
.defaultValue(classOf[SlashEncodedDayPartitionValueExtractor].getCanonicalName)
|
||||
.withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, "
|
||||
+ "default 'SlashEncodedDayPartitionValueExtractor'.")
|
||||
.withInferFunction(hivePartitionExtractorInferFunc)
|
||||
|
||||
val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.assume_date_partitioning")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Assume partitioning is yyyy/mm/dd")
|
||||
|
||||
val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.use_pre_apache_input_format")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. "
|
||||
+ "Use this when you are in the process of migrating from "
|
||||
+ "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format")
|
||||
/**
|
||||
* @deprecated Hive Specific Configs are moved to {@link HiveSyncConfig}
|
||||
*/
|
||||
@Deprecated
|
||||
val HIVE_SYNC_ENABLED: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_ENABLED
|
||||
@Deprecated
|
||||
val META_SYNC_ENABLED: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ENABLED
|
||||
@Deprecated
|
||||
val HIVE_DATABASE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_DATABASE_NAME
|
||||
@Deprecated
|
||||
val hiveTableOptKeyInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.TABLE_NAME_INFERENCE_FUNCTION
|
||||
@Deprecated
|
||||
val HIVE_TABLE: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_TABLE_NAME
|
||||
@Deprecated
|
||||
val HIVE_BASE_FILE_FORMAT: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT
|
||||
@Deprecated
|
||||
val HIVE_USER: ConfigProperty[String] = HiveSyncConfig.HIVE_USER
|
||||
@Deprecated
|
||||
val HIVE_PASS: ConfigProperty[String] = HiveSyncConfig.HIVE_PASS
|
||||
@Deprecated
|
||||
val HIVE_URL: ConfigProperty[String] = HiveSyncConfig.HIVE_URL
|
||||
@Deprecated
|
||||
val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfig.METASTORE_URIS
|
||||
@Deprecated
|
||||
val hivePartitionFieldsInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_FIELDS_INFERENCE_FUNCTION
|
||||
@Deprecated
|
||||
val HIVE_PARTITION_FIELDS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS
|
||||
@Deprecated
|
||||
val hivePartitionExtractorInferFunc: JavaFunction[HoodieConfig, Option[String]] = HoodieSyncConfig.PARTITION_EXTRACTOR_CLASS_FUNCTION
|
||||
@Deprecated
|
||||
val HIVE_PARTITION_EXTRACTOR_CLASS: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS
|
||||
@Deprecated
|
||||
val HIVE_ASSUME_DATE_PARTITION: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION
|
||||
@Deprecated
|
||||
val HIVE_USE_PRE_APACHE_INPUT_FORMAT: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT
|
||||
|
||||
/** @deprecated Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */
|
||||
@Deprecated
|
||||
val HIVE_USE_JDBC: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.use_jdbc")
|
||||
.defaultValue("true")
|
||||
.deprecatedAfter("0.9.0")
|
||||
.withDocumentation("Use JDBC when hive synchronization is enabled")
|
||||
|
||||
val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.auto_create_database")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Auto create hive database if does not exists")
|
||||
|
||||
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.ignore_exceptions")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("")
|
||||
|
||||
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.skip_ro_suffix")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Skip the _ro suffix for Read optimized table, when registering")
|
||||
|
||||
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.support_timestamp")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. " +
|
||||
"Disabled by default for backward compatibility.")
|
||||
val HIVE_USE_JDBC: ConfigProperty[String] = HiveSyncConfig.HIVE_USE_JDBC
|
||||
@Deprecated
|
||||
val HIVE_AUTO_CREATE_DATABASE: ConfigProperty[String] = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE
|
||||
@Deprecated
|
||||
val HIVE_IGNORE_EXCEPTIONS: ConfigProperty[String] = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS
|
||||
@Deprecated
|
||||
val HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE
|
||||
@Deprecated
|
||||
val HIVE_SUPPORT_TIMESTAMP_TYPE: ConfigProperty[String] = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE
|
||||
|
||||
/**
|
||||
* Flag to indicate whether to use conditional syncing in HiveSync.
|
||||
* If set true, the Hive sync procedure will only run if partition or schema changes are detected.
|
||||
* By default true.
|
||||
*/
|
||||
val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.conditional_sync")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Enables conditional hive sync, where partition or schema change must exist to perform sync to hive.")
|
||||
|
||||
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.table_properties")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Additional properties to store with table.")
|
||||
|
||||
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.serde_properties")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Serde properties to hive table.")
|
||||
|
||||
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.sync_as_datasource")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("")
|
||||
@Deprecated
|
||||
val HIVE_CONDITIONAL_SYNC: ConfigProperty[String] = HoodieSyncConfig.META_SYNC_CONDITIONAL_SYNC
|
||||
@Deprecated
|
||||
val HIVE_TABLE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_PROPERTIES
|
||||
@Deprecated
|
||||
val HIVE_TABLE_SERDE_PROPERTIES: ConfigProperty[String] = HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES
|
||||
@Deprecated
|
||||
val HIVE_SYNC_AS_DATA_SOURCE_TABLE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE
|
||||
|
||||
// Create table as managed table
|
||||
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[Boolean] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.create_managed_table")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Whether to sync the table as managed table.")
|
||||
|
||||
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[Int] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.batch_num")
|
||||
.defaultValue(1000)
|
||||
.withDocumentation("The number of partitions one batch when synchronous partitions to hive.")
|
||||
|
||||
val HIVE_SYNC_MODE: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.mode")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.")
|
||||
|
||||
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[Boolean] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.bucket_sync")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Whether sync hive metastore bucket specification when using bucket index." +
|
||||
"The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'")
|
||||
|
||||
val HIVE_SYNC_COMMENT: ConfigProperty[String] = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.sync_comment")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Whether to sync the table column comments while syncing the table.")
|
||||
@Deprecated
|
||||
val HIVE_CREATE_MANAGED_TABLE: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE
|
||||
@Deprecated
|
||||
val HIVE_BATCH_SYNC_PARTITION_NUM: ConfigProperty[java.lang.Integer] = HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM
|
||||
@Deprecated
|
||||
val HIVE_SYNC_MODE: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_MODE
|
||||
@Deprecated
|
||||
val HIVE_SYNC_BUCKET_SYNC: ConfigProperty[java.lang.Boolean] = HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC
|
||||
@Deprecated
|
||||
val HIVE_SYNC_COMMENT: ConfigProperty[String] = HiveSyncConfig.HIVE_SYNC_COMMENT;
|
||||
|
||||
// Async Compaction - Enabled by default for MOR
|
||||
val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty
|
||||
@@ -584,19 +479,19 @@ object DataSourceWriteOptions {
|
||||
|
||||
/** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HIVE_ASSUME_DATE_PARTITION.key()
|
||||
val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key()
|
||||
/** @deprecated Use {@link HIVE_USE_PRE_APACHE_INPUT_FORMAT} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
|
||||
val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key()
|
||||
/** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_USE_JDBC_OPT_KEY = HIVE_USE_JDBC.key()
|
||||
val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key()
|
||||
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.key()
|
||||
val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key()
|
||||
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.key()
|
||||
val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key()
|
||||
/** @deprecated Use {@link STREAMING_IGNORE_FAILED_BATCH} and its methods instead */
|
||||
@Deprecated
|
||||
val STREAMING_IGNORE_FAILED_BATCH_OPT_KEY = STREAMING_IGNORE_FAILED_BATCH.key()
|
||||
@@ -611,34 +506,34 @@ object DataSourceWriteOptions {
|
||||
val DEFAULT_META_SYNC_CLIENT_TOOL_CLASS = META_SYNC_CLIENT_TOOL_CLASS_NAME.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key()
|
||||
val HIVE_SYNC_ENABLED_OPT_KEY = HiveSyncConfig.HIVE_SYNC_ENABLED.key()
|
||||
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
|
||||
@Deprecated
|
||||
val META_SYNC_ENABLED_OPT_KEY = META_SYNC_ENABLED.key()
|
||||
val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
|
||||
/** @deprecated Use {@link HIVE_DATABASE} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_DATABASE_OPT_KEY = HIVE_DATABASE.key()
|
||||
val HIVE_DATABASE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
|
||||
/** @deprecated Use {@link HIVE_TABLE} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_TABLE_OPT_KEY = HIVE_TABLE.key()
|
||||
val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()
|
||||
/** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_BASE_FILE_FORMAT_OPT_KEY = HIVE_BASE_FILE_FORMAT.key()
|
||||
val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key()
|
||||
/** @deprecated Use {@link HIVE_USER} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_USER_OPT_KEY = HIVE_USER.key()
|
||||
val HIVE_USER_OPT_KEY = HiveSyncConfig.HIVE_USER.key()
|
||||
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_PASS_OPT_KEY = HIVE_PASS.key()
|
||||
val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key()
|
||||
/** @deprecated Use {@link HIVE_URL} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_URL_OPT_KEY = HIVE_URL.key()
|
||||
val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key()
|
||||
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_PARTITION_FIELDS_OPT_KEY = HIVE_PARTITION_FIELDS.key()
|
||||
val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key()
|
||||
/** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HIVE_PARTITION_EXTRACTOR_CLASS.key()
|
||||
val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key()
|
||||
|
||||
/** @deprecated Use {@link KEYGENERATOR_CLASS_NAME} and its methods instead */
|
||||
@Deprecated
|
||||
@@ -748,60 +643,60 @@ object DataSourceWriteOptions {
|
||||
|
||||
/** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HIVE_SYNC_ENABLED.defaultValue()
|
||||
val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue()
|
||||
/** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = META_SYNC_ENABLED.defaultValue()
|
||||
val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_DATABASE} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_DATABASE_OPT_VAL = HIVE_DATABASE.defaultValue()
|
||||
val DEFAULT_HIVE_DATABASE_OPT_VAL = HoodieSyncConfig.META_SYNC_DATABASE_NAME.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_TABLE} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_TABLE_OPT_VAL = HIVE_TABLE.defaultValue()
|
||||
val DEFAULT_HIVE_TABLE_OPT_VAL = HoodieSyncConfig.META_SYNC_TABLE_NAME.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HIVE_BASE_FILE_FORMAT.defaultValue()
|
||||
val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_USER} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_USER_OPT_VAL = HIVE_USER.defaultValue()
|
||||
val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfig.HIVE_USER.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_PASS} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_PASS_OPT_VAL = HIVE_PASS.defaultValue()
|
||||
val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_URL} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_URL_OPT_VAL = HIVE_URL.defaultValue()
|
||||
val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HIVE_PARTITION_FIELDS.defaultValue()
|
||||
val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HIVE_PARTITION_EXTRACTOR_CLASS.defaultValue()
|
||||
val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HIVE_ASSUME_DATE_PARTITION.defaultValue()
|
||||
val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue()
|
||||
@Deprecated
|
||||
val DEFAULT_USE_PRE_APACHE_INPUT_FORMAT_OPT_VAL = "false"
|
||||
/** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HIVE_USE_JDBC.defaultValue()
|
||||
val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HiveSyncConfig.HIVE_USE_JDBC.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.defaultValue()
|
||||
val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HIVE_IGNORE_EXCEPTIONS.defaultValue()
|
||||
val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
|
||||
val HIVE_SKIP_RO_SUFFIX = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key()
|
||||
/** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
|
||||
val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue()
|
||||
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
|
||||
@Deprecated
|
||||
val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key()
|
||||
val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key()
|
||||
/** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */
|
||||
@Deprecated
|
||||
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
|
||||
val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue()
|
||||
/** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */
|
||||
@Deprecated
|
||||
val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key()
|
||||
|
||||
@@ -19,7 +19,6 @@ package org.apache.hudi
|
||||
|
||||
import org.apache.avro.Schema
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.avro.reflect.AvroSchema
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.{FileSystem, Path}
|
||||
import org.apache.hadoop.hive.conf.HiveConf
|
||||
@@ -32,7 +31,7 @@ import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.model._
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline
|
||||
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
|
||||
import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils}
|
||||
import org.apache.hudi.common.util.{CommitUtils, StringUtils}
|
||||
import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME}
|
||||
import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig}
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
@@ -40,24 +39,21 @@ import org.apache.hudi.execution.bulkinsert.{BulkInsertInternalPartitionerWithRo
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, HiveSyncTool}
|
||||
import org.apache.hudi.index.SparkHoodieIndexFactory
|
||||
import org.apache.hudi.internal.DataSourceInternalWriterHelper
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool
|
||||
import org.apache.hudi.keygen.{TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator}
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.apache.hudi.sync.common.util.SyncUtilHelpers
|
||||
import org.apache.hudi.table.BulkInsertPartitioner
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.SPARK_VERSION
|
||||
import org.apache.spark.api.java.JavaSparkContext
|
||||
import org.apache.spark.sql.hive.HiveExternalCatalog
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf}
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.SparkContext
|
||||
import org.apache.spark.sql.internal.StaticSQLConf
|
||||
import org.apache.spark.sql.types.StructType
|
||||
import org.apache.spark.{SPARK_VERSION, SparkContext}
|
||||
|
||||
import java.util.Properties
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.mutable
|
||||
import scala.collection.mutable.ListBuffer
|
||||
|
||||
object HoodieSparkSqlWriter {
|
||||
|
||||
@@ -512,14 +508,7 @@ object HoodieSparkSqlWriter {
|
||||
+ " To use row writer please switch to spark 2 or spark 3")
|
||||
}
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params)
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
|
||||
val metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
|
||||
val syncHiveSuccess =
|
||||
if (hiveSyncEnabled || metaSyncEnabled) {
|
||||
metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
} else {
|
||||
true
|
||||
}
|
||||
val syncHiveSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema)
|
||||
(syncHiveSuccess, common.util.Option.ofNullable(instantTime))
|
||||
}
|
||||
|
||||
@@ -558,57 +547,10 @@ object HoodieSparkSqlWriter {
|
||||
}
|
||||
}
|
||||
|
||||
private def syncHive(basePath: Path, fs: FileSystem, hoodieConfig: HoodieConfig, sqlConf: SQLConf): Boolean = {
|
||||
val hiveSyncConfig: HiveSyncConfig = buildSyncConfig(basePath, hoodieConfig, sqlConf)
|
||||
val hiveConf: HiveConf = new HiveConf()
|
||||
hiveConf.addResource(fs.getConf)
|
||||
if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) {
|
||||
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris)
|
||||
}
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable()
|
||||
true
|
||||
}
|
||||
|
||||
private def buildSyncConfig(basePath: Path, hoodieConfig: HoodieConfig, sqlConf: SQLConf): HiveSyncConfig = {
|
||||
val hiveSyncConfig: HiveSyncConfig = new HiveSyncConfig()
|
||||
hiveSyncConfig.basePath = basePath.toString
|
||||
hiveSyncConfig.baseFileFormat = hoodieConfig.getString(HIVE_BASE_FILE_FORMAT)
|
||||
hiveSyncConfig.usePreApacheInputFormat =
|
||||
hoodieConfig.getStringOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT).toBoolean
|
||||
hiveSyncConfig.databaseName = hoodieConfig.getString(HIVE_DATABASE)
|
||||
hiveSyncConfig.tableName = hoodieConfig.getString(HIVE_TABLE)
|
||||
hiveSyncConfig.hiveUser = hoodieConfig.getString(HIVE_USER)
|
||||
hiveSyncConfig.hivePass = hoodieConfig.getString(HIVE_PASS)
|
||||
hiveSyncConfig.jdbcUrl = hoodieConfig.getString(HIVE_URL)
|
||||
hiveSyncConfig.metastoreUris = hoodieConfig.getStringOrDefault(METASTORE_URIS)
|
||||
hiveSyncConfig.skipROSuffix = hoodieConfig.getStringOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE,
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue).toBoolean
|
||||
hiveSyncConfig.partitionFields =
|
||||
ListBuffer(hoodieConfig.getString(HIVE_PARTITION_FIELDS).split(",").map(_.trim).filter(!_.isEmpty).toList: _*)
|
||||
hiveSyncConfig.partitionValueExtractorClass = hoodieConfig.getString(HIVE_PARTITION_EXTRACTOR_CLASS)
|
||||
hiveSyncConfig.useJdbc = hoodieConfig.getBoolean(HIVE_USE_JDBC)
|
||||
hiveSyncConfig.useFileListingFromMetadata = hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE)
|
||||
hiveSyncConfig.ignoreExceptions = hoodieConfig.getStringOrDefault(HIVE_IGNORE_EXCEPTIONS).toBoolean
|
||||
hiveSyncConfig.supportTimestamp = hoodieConfig.getStringOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE).toBoolean
|
||||
hiveSyncConfig.autoCreateDatabase = hoodieConfig.getStringOrDefault(HIVE_AUTO_CREATE_DATABASE).toBoolean
|
||||
hiveSyncConfig.decodePartition = hoodieConfig.getStringOrDefault(URL_ENCODE_PARTITIONING).toBoolean
|
||||
hiveSyncConfig.batchSyncNum = hoodieConfig.getStringOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM).toInt
|
||||
|
||||
hiveSyncConfig.syncAsSparkDataSourceTable = hoodieConfig.getStringOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE).toBoolean
|
||||
hiveSyncConfig.sparkSchemaLengthThreshold = sqlConf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD)
|
||||
hiveSyncConfig.createManagedTable = hoodieConfig.getBoolean(HIVE_CREATE_MANAGED_TABLE)
|
||||
hiveSyncConfig.syncMode = hoodieConfig.getString(HIVE_SYNC_MODE)
|
||||
hiveSyncConfig.serdeProperties = hoodieConfig.getString(HIVE_TABLE_SERDE_PROPERTIES)
|
||||
hiveSyncConfig.tableProperties = hoodieConfig.getString(HIVE_TABLE_PROPERTIES)
|
||||
hiveSyncConfig.sparkVersion = SPARK_VERSION
|
||||
hiveSyncConfig.syncComment = hoodieConfig.getStringOrDefault(HIVE_SYNC_COMMENT).toBoolean
|
||||
hiveSyncConfig
|
||||
}
|
||||
|
||||
private def metaSync(spark: SparkSession, hoodieConfig: HoodieConfig, basePath: Path,
|
||||
schema: StructType): Boolean = {
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean
|
||||
var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean
|
||||
val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean
|
||||
var metaSyncEnabled = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_ENABLED).toBoolean
|
||||
var syncClientToolClassSet = scala.collection.mutable.Set[String]()
|
||||
hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass)
|
||||
|
||||
@@ -617,29 +559,23 @@ object HoodieSparkSqlWriter {
|
||||
metaSyncEnabled = true
|
||||
syncClientToolClassSet += classOf[HiveSyncTool].getName
|
||||
}
|
||||
var metaSyncSuccess = true
|
||||
|
||||
if (metaSyncEnabled) {
|
||||
val fs = basePath.getFileSystem(spark.sessionState.newHadoopConf())
|
||||
val properties = new TypedProperties()
|
||||
properties.putAll(hoodieConfig.getProps)
|
||||
properties.put(HiveSyncConfig.HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD.key, spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD).toString)
|
||||
properties.put(HoodieSyncConfig.META_SYNC_SPARK_VERSION.key, SPARK_VERSION)
|
||||
properties.put(HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA.key, hoodieConfig.getBoolean(HoodieMetadataConfig.ENABLE))
|
||||
|
||||
val hiveConf: HiveConf = new HiveConf()
|
||||
hiveConf.addResource(fs.getConf)
|
||||
|
||||
syncClientToolClassSet.foreach(impl => {
|
||||
val syncSuccess = impl.trim match {
|
||||
case "org.apache.hudi.hive.HiveSyncTool" => {
|
||||
log.info("Syncing to Hive Metastore (URL: " + hoodieConfig.getString(HIVE_URL) + ")")
|
||||
syncHive(basePath, fs, hoodieConfig, spark.sessionState.conf)
|
||||
true
|
||||
}
|
||||
case _ => {
|
||||
val properties = new Properties()
|
||||
properties.putAll(hoodieConfig.getProps)
|
||||
properties.put("basePath", basePath.toString)
|
||||
val syncHoodie = ReflectionUtils.loadClass(impl.trim, Array[Class[_]](classOf[Properties], classOf[FileSystem]), properties, fs).asInstanceOf[AbstractSyncTool]
|
||||
syncHoodie.syncHoodieTable()
|
||||
true
|
||||
}
|
||||
}
|
||||
metaSyncSuccess = metaSyncSuccess && syncSuccess
|
||||
SyncUtilHelpers.runHoodieMetaSync(impl.trim, properties, hiveConf, fs, basePath.toString, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue)
|
||||
})
|
||||
}
|
||||
metaSyncSuccess
|
||||
true
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -18,13 +18,14 @@
|
||||
package org.apache.hudi
|
||||
|
||||
import java.util.Properties
|
||||
|
||||
import org.apache.hudi.DataSourceOptionsHelper.allAlternatives
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig.ENABLE
|
||||
import org.apache.hudi.common.config.{DFSPropertiesConfiguration, HoodieConfig, TypedProperties}
|
||||
import org.apache.hudi.common.table.HoodieTableConfig
|
||||
import org.apache.hudi.exception.HoodieException
|
||||
import org.apache.hudi.hive.HiveSyncConfig
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.apache.spark.sql.SparkSession
|
||||
import org.apache.spark.sql.hudi.command.SqlKeyGenerator
|
||||
|
||||
@@ -64,21 +65,21 @@ object HoodieWriterUtils {
|
||||
hoodieConfig.setDefaultValue(STREAMING_RETRY_INTERVAL_MS)
|
||||
hoodieConfig.setDefaultValue(STREAMING_IGNORE_FAILED_BATCH)
|
||||
hoodieConfig.setDefaultValue(META_SYNC_CLIENT_TOOL_CLASS_NAME)
|
||||
hoodieConfig.setDefaultValue(HIVE_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(META_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(HIVE_DATABASE)
|
||||
hoodieConfig.setDefaultValue(HIVE_TABLE)
|
||||
hoodieConfig.setDefaultValue(HIVE_BASE_FILE_FORMAT)
|
||||
hoodieConfig.setDefaultValue(HIVE_USER)
|
||||
hoodieConfig.setDefaultValue(HIVE_PASS)
|
||||
hoodieConfig.setDefaultValue(HIVE_URL)
|
||||
hoodieConfig.setDefaultValue(METASTORE_URIS)
|
||||
hoodieConfig.setDefaultValue(HIVE_PARTITION_FIELDS)
|
||||
hoodieConfig.setDefaultValue(HIVE_PARTITION_EXTRACTOR_CLASS)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_ENABLED)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_DATABASE_NAME)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_TABLE_NAME)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.METASTORE_URIS)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USER)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_PASS)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_URL)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS)
|
||||
hoodieConfig.setDefaultValue(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS)
|
||||
hoodieConfig.setDefaultValue(HIVE_STYLE_PARTITIONING)
|
||||
hoodieConfig.setDefaultValue(HIVE_USE_JDBC)
|
||||
hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE)
|
||||
hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE)
|
||||
hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE)
|
||||
hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE)
|
||||
hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE)
|
||||
hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE)
|
||||
|
||||
@@ -22,8 +22,9 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext
|
||||
import org.apache.hudi.common.fs.FSUtils
|
||||
import org.apache.hudi.common.util.PartitionPathEncodeUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor}
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.apache.hudi.{DataSourceWriteOptions, HoodieSparkSqlWriter}
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
import org.apache.spark.sql.catalyst.analysis.Resolver
|
||||
@@ -102,15 +103,15 @@ case class AlterHoodieTableDropPartitionCommand(
|
||||
RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","),
|
||||
PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""),
|
||||
PARTITIONPATH_FIELD.key -> partitionFields,
|
||||
HIVE_SYNC_ENABLED.key -> enableHive.toString,
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_USE_JDBC.key -> "false",
|
||||
HIVE_DATABASE.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
|
||||
HIVE_TABLE.key -> hoodieCatalogTable.table.identifier.table,
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_PARTITION_FIELDS.key -> partitionFields,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName
|
||||
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HiveSyncConfig.HIVE_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HiveSyncConfig.HIVE_USE_JDBC.key -> "false",
|
||||
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> hoodieCatalogTable.table.identifier.database.getOrElse("default"),
|
||||
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> hoodieCatalogTable.table.identifier.table,
|
||||
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> partitionFields,
|
||||
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command
|
||||
import org.apache.hadoop.conf.Configuration
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions
|
||||
import org.apache.hudi.hive.HiveSyncConfig
|
||||
import org.apache.hudi.hive.util.ConfigUtils
|
||||
import org.apache.hudi.sql.InsertMode
|
||||
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, HoodieCatalogTable}
|
||||
@@ -80,9 +81,9 @@ case class CreateHoodieTableAsSelectCommand(
|
||||
|
||||
val tblProperties = hoodieCatalogTable.catalogProperties
|
||||
val options = Map(
|
||||
DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
|
||||
DataSourceWriteOptions.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
|
||||
DataSourceWriteOptions.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava),
|
||||
HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key -> (table.tableType == CatalogTableType.MANAGED).toString,
|
||||
HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key -> ConfigUtils.configToString(tblProperties.asJava),
|
||||
HiveSyncConfig.HIVE_TABLE_PROPERTIES.key -> ConfigUtils.configToString(table.properties.asJava),
|
||||
DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(),
|
||||
DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true"
|
||||
)
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.spark.sql.hudi.command
|
||||
import org.apache.hudi.DataSourceWriteOptions.{OPERATION, _}
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.HiveSyncConfig
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.{DataSourceWriteOptions, SparkAdapterSupport}
|
||||
import org.apache.spark.sql._
|
||||
@@ -75,8 +76,8 @@ case class DeleteHoodieTableCommand(deleteTable: DeleteFromTable) extends Hoodie
|
||||
SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName,
|
||||
OPERATION.key -> DataSourceWriteOptions.DELETE_OPERATION_OPT_VAL,
|
||||
PARTITIONPATH_FIELD.key -> tableConfig.getPartitionFieldProp,
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
|
||||
SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL
|
||||
)
|
||||
|
||||
@@ -22,8 +22,9 @@ import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.util.StringUtils
|
||||
import org.apache.hudi.config.HoodieWriteConfig
|
||||
import org.apache.hudi.config.HoodieWriteConfig.TBL_NAME
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor
|
||||
import org.apache.hudi.hive.{HiveSyncConfig, MultiPartKeysValueExtractor}
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, SparkAdapterSupport}
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.catalyst.TableIdentifier
|
||||
@@ -462,14 +463,14 @@ case class MergeIntoHoodieTableCommand(mergeInto: MergeIntoTable) extends Hoodie
|
||||
URL_ENCODE_PARTITIONING.key -> tableConfig.getUrlEncodePartitioning,
|
||||
KEYGENERATOR_CLASS_NAME.key -> classOf[SqlKeyGenerator].getCanonicalName,
|
||||
SqlKeyGenerator.ORIGIN_KEYGEN_CLASS_NAME -> tableConfig.getKeyGeneratorClassName,
|
||||
META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HIVE_USE_JDBC.key -> "false",
|
||||
HIVE_DATABASE.key -> targetTableDb,
|
||||
HIVE_TABLE.key -> targetTableName,
|
||||
HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HIVE_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
|
||||
HIVE_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
HoodieSyncConfig.META_SYNC_ENABLED.key -> enableHive.toString,
|
||||
HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(),
|
||||
HiveSyncConfig.HIVE_USE_JDBC.key -> "false",
|
||||
HoodieSyncConfig.META_SYNC_DATABASE_NAME.key -> targetTableDb,
|
||||
HoodieSyncConfig.META_SYNC_TABLE_NAME.key -> targetTableName,
|
||||
HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true",
|
||||
HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key -> tableConfig.getPartitionFieldProp,
|
||||
HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key -> classOf[MultiPartKeysValueExtractor].getCanonicalName,
|
||||
HoodieWriteConfig.INSERT_PARALLELISM_VALUE.key -> "200", // set the default parallelism to 200 for sql
|
||||
HoodieWriteConfig.UPSERT_PARALLELISM_VALUE.key -> "200",
|
||||
HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200",
|
||||
|
||||
@@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
@@ -255,24 +256,24 @@ public class HoodieJavaApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.NonPartitionedExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
@@ -125,24 +126,24 @@ public class HoodieJavaGenerateApp {
|
||||
private DataFrameWriter<Row> updateHiveSyncConfig(DataFrameWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (nonPartitionedTable) {
|
||||
writer = writer
|
||||
.option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
.option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
NonPartitionedExtractor.class.getCanonicalName())
|
||||
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), "");
|
||||
} else if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -27,6 +27,7 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.TableNotFoundException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
|
||||
@@ -382,19 +383,19 @@ public class HoodieJavaStreamingApp {
|
||||
private DataStreamWriter<Row> updateHiveSyncConfig(DataStreamWriter<Row> writer) {
|
||||
if (enableHiveSync) {
|
||||
LOG.info("Enabling Hive sync to " + hiveJdbcUrl);
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_TABLE().key(), hiveTable)
|
||||
.option(DataSourceWriteOptions.HIVE_DATABASE().key(), hiveDB)
|
||||
.option(DataSourceWriteOptions.HIVE_URL().key(), hiveJdbcUrl)
|
||||
.option(DataSourceWriteOptions.HIVE_USER().key(), hiveUser)
|
||||
.option(DataSourceWriteOptions.HIVE_PASS().key(), hivePass)
|
||||
.option(DataSourceWriteOptions.HIVE_SYNC_ENABLED().key(), "true");
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), hiveTable)
|
||||
.option(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), hiveDB)
|
||||
.option(HiveSyncConfig.HIVE_URL.key(), hiveJdbcUrl)
|
||||
.option(HiveSyncConfig.HIVE_USER.key(), hiveUser)
|
||||
.option(HiveSyncConfig.HIVE_PASS.key(), hivePass)
|
||||
.option(HiveSyncConfig.HIVE_SYNC_ENABLED.key(), "true");
|
||||
if (useMultiPartitionKeys) {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getCanonicalName());
|
||||
} else {
|
||||
writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option(
|
||||
DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option(
|
||||
HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
SlashEncodedDayPartitionValueExtractor.class.getCanonicalName());
|
||||
}
|
||||
}
|
||||
|
||||
@@ -20,6 +20,7 @@ package org.apache.hudi
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, MultiPartKeysValueExtractor}
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, SimpleKeyGenerator}
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig
|
||||
import org.junit.jupiter.api.Assertions.assertEquals
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
@@ -31,10 +32,10 @@ class TestDataSourceOptions {
|
||||
)
|
||||
val modifiedOptions1 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions1)
|
||||
assertEquals(classOf[ComplexKeyGenerator].getName, modifiedOptions1(KEYGENERATOR_CLASS_NAME.key))
|
||||
assertEquals("hudi_table", modifiedOptions1(HIVE_TABLE.key))
|
||||
assertEquals("year,month", modifiedOptions1(HIVE_PARTITION_FIELDS.key))
|
||||
assertEquals("hudi_table", modifiedOptions1(HoodieSyncConfig.META_SYNC_TABLE_NAME.key))
|
||||
assertEquals("year,month", modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key))
|
||||
assertEquals(classOf[MultiPartKeysValueExtractor].getName,
|
||||
modifiedOptions1(HIVE_PARTITION_EXTRACTOR_CLASS.key))
|
||||
modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key))
|
||||
|
||||
val inputOptions2 = Map(
|
||||
TABLE_NAME.key -> "hudi_table",
|
||||
@@ -43,9 +44,9 @@ class TestDataSourceOptions {
|
||||
)
|
||||
val modifiedOptions2 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions2)
|
||||
assertEquals(classOf[SimpleKeyGenerator].getName, modifiedOptions2(KEYGENERATOR_CLASS_NAME.key))
|
||||
assertEquals("hudi_table", modifiedOptions2(HIVE_TABLE.key))
|
||||
assertEquals("year", modifiedOptions2(HIVE_PARTITION_FIELDS.key))
|
||||
assertEquals("hudi_table", modifiedOptions2(HoodieSyncConfig.META_SYNC_TABLE_NAME.key))
|
||||
assertEquals("year", modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key))
|
||||
assertEquals(classOf[HiveStylePartitionValueExtractor].getName,
|
||||
modifiedOptions2(HIVE_PARTITION_EXTRACTOR_CLASS.key))
|
||||
modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -664,55 +664,6 @@ class TestHoodieSparkSqlWriter {
|
||||
assertEquals(expectedSchema, actualSchema)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for build sync config for spark sql.
|
||||
*/
|
||||
@Test
|
||||
def testBuildSyncConfigForSparkSql(): Unit = {
|
||||
val params = Map(
|
||||
"path" -> tempBasePath,
|
||||
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition",
|
||||
DataSourceWriteOptions.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key -> "true",
|
||||
DataSourceWriteOptions.HIVE_CREATE_MANAGED_TABLE.key -> "true"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[HoodieConfig], classOf[SQLConf])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig]
|
||||
assertTrue(hiveSyncConfig.skipROSuffix)
|
||||
assertTrue(hiveSyncConfig.createManagedTable)
|
||||
assertTrue(hiveSyncConfig.syncAsSparkDataSourceTable)
|
||||
assertResult(spark.sessionState.conf.getConf(StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD))(hiveSyncConfig.sparkSchemaLengthThreshold)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for build sync config for skip Ro Suffix values.
|
||||
*/
|
||||
@Test
|
||||
def testBuildSyncConfigForSkipRoSuffixValues(): Unit = {
|
||||
val params = Map(
|
||||
"path" -> tempBasePath,
|
||||
DataSourceWriteOptions.TABLE_NAME.key -> "test_hoodie",
|
||||
DataSourceWriteOptions.HIVE_PARTITION_FIELDS.key -> "partition"
|
||||
)
|
||||
val parameters = HoodieWriterUtils.parametersWithWriteDefaults(params)
|
||||
val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(parameters)
|
||||
val buildSyncConfigMethod =
|
||||
HoodieSparkSqlWriter.getClass.getDeclaredMethod("buildSyncConfig", classOf[Path],
|
||||
classOf[HoodieConfig], classOf[SQLConf])
|
||||
buildSyncConfigMethod.setAccessible(true)
|
||||
val hiveSyncConfig = buildSyncConfigMethod.invoke(HoodieSparkSqlWriter,
|
||||
new Path(tempBasePath), hoodieConfig, spark.sessionState.conf).asInstanceOf[HiveSyncConfig]
|
||||
assertFalse(hiveSyncConfig.skipROSuffix)
|
||||
}
|
||||
|
||||
/**
|
||||
* Test case for incremental view with replacement.
|
||||
*/
|
||||
|
||||
@@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat;
|
||||
import org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
@@ -41,7 +43,6 @@ import org.apache.parquet.schema.MessageType;
|
||||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
@@ -63,8 +64,8 @@ public class DLASyncTool extends AbstractSyncTool {
|
||||
private final String snapshotTableName;
|
||||
private final Option<String> roTableTableName;
|
||||
|
||||
public DLASyncTool(Properties properties, FileSystem fs) {
|
||||
super(properties, fs);
|
||||
public DLASyncTool(TypedProperties properties, Configuration conf, FileSystem fs) {
|
||||
super(properties, conf, fs);
|
||||
this.hoodieDLAClient = new HoodieDLAClient(Utils.propertiesToConfig(properties), fs);
|
||||
this.cfg = Utils.propertiesToConfig(properties);
|
||||
switch (hoodieDLAClient.getTableType()) {
|
||||
@@ -205,7 +206,8 @@ public class DLASyncTool extends AbstractSyncTool {
|
||||
cmd.usage();
|
||||
System.exit(1);
|
||||
}
|
||||
FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration());
|
||||
new DLASyncTool(Utils.configToProperties(cfg), fs).syncHoodieTable();
|
||||
Configuration hadoopConf = new Configuration();
|
||||
FileSystem fs = FSUtils.getFs(cfg.basePath, hadoopConf);
|
||||
new DLASyncTool(Utils.configToProperties(cfg), hadoopConf, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,12 +18,12 @@
|
||||
|
||||
package org.apache.hudi.dla.util;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.dla.DLASyncConfig;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Properties;
|
||||
|
||||
public class Utils {
|
||||
public static String DLA_DATABASE_OPT_KEY = "hoodie.datasource.dla_sync.database";
|
||||
@@ -39,8 +39,8 @@ public class Utils {
|
||||
public static String DLA_SKIP_RT_SYNC = "hoodie.datasource.dla_sync.skip_rt_sync";
|
||||
public static String DLA_SYNC_HIVE_STYLE_PARTITIONING = "hoodie.datasource.dla_sync.hive.style.partitioning";
|
||||
|
||||
public static Properties configToProperties(DLASyncConfig cfg) {
|
||||
Properties properties = new Properties();
|
||||
public static TypedProperties configToProperties(DLASyncConfig cfg) {
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.put(DLA_DATABASE_OPT_KEY, cfg.databaseName);
|
||||
properties.put(DLA_TABLE_OPT_KEY, cfg.tableName);
|
||||
properties.put(DLA_USER_OPT_KEY, cfg.dlaUser);
|
||||
@@ -54,7 +54,7 @@ public class Utils {
|
||||
return properties;
|
||||
}
|
||||
|
||||
public static DLASyncConfig propertiesToConfig(Properties properties) {
|
||||
public static DLASyncConfig propertiesToConfig(TypedProperties properties) {
|
||||
DLASyncConfig config = new DLASyncConfig();
|
||||
config.databaseName = properties.getProperty(DLA_DATABASE_OPT_KEY);
|
||||
config.tableName = properties.getProperty(DLA_TABLE_OPT_KEY);
|
||||
|
||||
@@ -18,27 +18,16 @@
|
||||
|
||||
package org.apache.hudi.hive;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Configs needed to sync data into Hive.
|
||||
* Configs needed to sync data into the Hive Metastore.
|
||||
*/
|
||||
public class HiveSyncConfig implements Serializable {
|
||||
|
||||
@Parameter(names = {"--database"}, description = "name of the target database in Hive", required = true)
|
||||
public String databaseName;
|
||||
|
||||
@Parameter(names = {"--table"}, description = "name of the target table in Hive", required = true)
|
||||
public String tableName;
|
||||
|
||||
@Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)")
|
||||
public String baseFileFormat = "PARQUET";
|
||||
public class HiveSyncConfig extends HoodieSyncConfig {
|
||||
|
||||
@Parameter(names = {"--user"}, description = "Hive username")
|
||||
public String hiveUser;
|
||||
@@ -52,48 +41,31 @@ public class HiveSyncConfig implements Serializable {
|
||||
@Parameter(names = {"--metastore-uris"}, description = "Hive metastore uris")
|
||||
public String metastoreUris;
|
||||
|
||||
@Parameter(names = {"--base-path"}, description = "Basepath of hoodie table to sync", required = true)
|
||||
public String basePath;
|
||||
|
||||
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by")
|
||||
public List<String> partitionFields = new ArrayList<>();
|
||||
|
||||
@Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor "
|
||||
+ "to extract the partition values from HDFS path")
|
||||
public String partitionValueExtractorClass = SlashEncodedDayPartitionValueExtractor.class.getName();
|
||||
|
||||
@Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"
|
||||
+ " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter")
|
||||
public Boolean assumeDatePartitioning = false;
|
||||
|
||||
@Parameter(names = {"--use-pre-apache-input-format"},
|
||||
description = "Use InputFormat under com.uber.hoodie package "
|
||||
+ "instead of org.apache.hudi package. Use this when you are in the process of migrating from "
|
||||
+ "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to "
|
||||
+ "org.apache.hudi input format.")
|
||||
public Boolean usePreApacheInputFormat = false;
|
||||
public Boolean usePreApacheInputFormat;
|
||||
|
||||
@Parameter(names = {"--bucket-spec"}, description = "bucket spec stored in metastore", required = false)
|
||||
public String bucketSpec;
|
||||
|
||||
@Deprecated
|
||||
@Parameter(names = {"--use-jdbc"}, description = "Hive jdbc connect url")
|
||||
public Boolean useJdbc = true;
|
||||
public Boolean useJdbc;
|
||||
|
||||
@Parameter(names = {"--sync-mode"}, description = "Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql")
|
||||
public String syncMode;
|
||||
|
||||
@Parameter(names = {"--auto-create-database"}, description = "Auto create hive database")
|
||||
public Boolean autoCreateDatabase = true;
|
||||
public Boolean autoCreateDatabase;
|
||||
|
||||
@Parameter(names = {"--ignore-exceptions"}, description = "Ignore hive exceptions")
|
||||
public Boolean ignoreExceptions = false;
|
||||
public Boolean ignoreExceptions;
|
||||
|
||||
@Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering")
|
||||
public Boolean skipROSuffix = false;
|
||||
|
||||
@Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
|
||||
public Boolean useFileListingFromMetadata = HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
|
||||
public Boolean skipROSuffix;
|
||||
|
||||
@Parameter(names = {"--table-properties"}, description = "Table properties to hive table")
|
||||
public String tableProperties;
|
||||
@@ -106,64 +78,170 @@ public class HiveSyncConfig implements Serializable {
|
||||
|
||||
@Parameter(names = {"--support-timestamp"}, description = "'INT64' with original type TIMESTAMP_MICROS is converted to hive 'timestamp' type."
|
||||
+ "Disabled by default for backward compatibility.")
|
||||
public Boolean supportTimestamp = false;
|
||||
|
||||
@Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing")
|
||||
public Boolean decodePartition = false;
|
||||
public Boolean supportTimestamp;
|
||||
|
||||
@Parameter(names = {"--managed-table"}, description = "Create a managed table")
|
||||
public Boolean createManagedTable = false;
|
||||
public Boolean createManagedTable;
|
||||
|
||||
@Parameter(names = {"--batch-sync-num"}, description = "The number of partitions one batch when synchronous partitions to hive")
|
||||
public Integer batchSyncNum = 1000;
|
||||
public Integer batchSyncNum;
|
||||
|
||||
@Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table.")
|
||||
public Boolean syncAsSparkDataSourceTable = true;
|
||||
public Boolean syncAsSparkDataSourceTable;
|
||||
|
||||
@Parameter(names = {"--spark-schema-length-threshold"}, description = "The maximum length allowed in a single cell when storing additional schema information in Hive's metastore.")
|
||||
public int sparkSchemaLengthThreshold = 4000;
|
||||
public int sparkSchemaLengthThreshold;
|
||||
|
||||
@Parameter(names = {"--with-operation-field"}, description = "Whether to include the '_hoodie_operation' field in the metadata fields")
|
||||
public Boolean withOperationField = false;
|
||||
|
||||
@Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.")
|
||||
public Boolean isConditionalSync = false;
|
||||
|
||||
@Parameter(names = {"--spark-version"}, description = "The spark version", required = false)
|
||||
public String sparkVersion;
|
||||
|
||||
@Parameter(names = {"--sync-comment"}, description = "synchronize table comments to hive")
|
||||
public boolean syncComment = false;
|
||||
|
||||
// enhance the similar function in child class
|
||||
public static HiveSyncConfig copy(HiveSyncConfig cfg) {
|
||||
HiveSyncConfig newConfig = new HiveSyncConfig();
|
||||
newConfig.basePath = cfg.basePath;
|
||||
newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning;
|
||||
newConfig.databaseName = cfg.databaseName;
|
||||
newConfig.hivePass = cfg.hivePass;
|
||||
newConfig.hiveUser = cfg.hiveUser;
|
||||
newConfig.partitionFields = cfg.partitionFields;
|
||||
newConfig.partitionValueExtractorClass = cfg.partitionValueExtractorClass;
|
||||
newConfig.jdbcUrl = cfg.jdbcUrl;
|
||||
newConfig.metastoreUris = cfg.metastoreUris;
|
||||
newConfig.tableName = cfg.tableName;
|
||||
newConfig.bucketSpec = cfg.bucketSpec;
|
||||
newConfig.usePreApacheInputFormat = cfg.usePreApacheInputFormat;
|
||||
newConfig.useFileListingFromMetadata = cfg.useFileListingFromMetadata;
|
||||
newConfig.supportTimestamp = cfg.supportTimestamp;
|
||||
newConfig.decodePartition = cfg.decodePartition;
|
||||
newConfig.tableProperties = cfg.tableProperties;
|
||||
newConfig.serdeProperties = cfg.serdeProperties;
|
||||
newConfig.createManagedTable = cfg.createManagedTable;
|
||||
newConfig.batchSyncNum = cfg.batchSyncNum;
|
||||
newConfig.syncAsSparkDataSourceTable = cfg.syncAsSparkDataSourceTable;
|
||||
newConfig.sparkSchemaLengthThreshold = cfg.sparkSchemaLengthThreshold;
|
||||
newConfig.withOperationField = cfg.withOperationField;
|
||||
newConfig.isConditionalSync = cfg.isConditionalSync;
|
||||
newConfig.sparkVersion = cfg.sparkVersion;
|
||||
newConfig.syncComment = cfg.syncComment;
|
||||
return newConfig;
|
||||
// HIVE SYNC SPECIFIC CONFIGS
|
||||
// NOTE: DO NOT USE uppercase for the keys as they are internally lower-cased. Using upper-cases causes
|
||||
// unexpected issues with config getting reset
|
||||
public static final ConfigProperty<String> HIVE_SYNC_ENABLED = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, register/sync the table to Apache Hive metastore.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_USER = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.username")
|
||||
.defaultValue("hive")
|
||||
.withDocumentation("hive user name to use");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_PASS = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.password")
|
||||
.defaultValue("hive")
|
||||
.withDocumentation("hive password to use");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_URL = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.jdbcurl")
|
||||
.defaultValue("jdbc:hive2://localhost:10000")
|
||||
.withDocumentation("Hive metastore url");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_USE_PRE_APACHE_INPUT_FORMAT = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.use_pre_apache_input_format")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. "
|
||||
+ "Use this when you are in the process of migrating from "
|
||||
+ "com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format");
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #HIVE_SYNC_MODE} instead of this config from 0.9.0
|
||||
*/
|
||||
@Deprecated
|
||||
public static final ConfigProperty<String> HIVE_USE_JDBC = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.use_jdbc")
|
||||
.defaultValue("true")
|
||||
.deprecatedAfter("0.9.0")
|
||||
.withDocumentation("Use JDBC when hive synchronization is enabled");
|
||||
|
||||
public static final ConfigProperty<String> METASTORE_URIS = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.metastore.uris")
|
||||
.defaultValue("thrift://localhost:9083")
|
||||
.withDocumentation("Hive metastore url");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_AUTO_CREATE_DATABASE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.auto_create_database")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Auto create hive database if does not exists");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_IGNORE_EXCEPTIONS = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.ignore_exceptions")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Ignore exceptions when syncing with Hive.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.skip_ro_suffix")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Skip the _ro suffix for Read optimized table, when registering");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SUPPORT_TIMESTAMP_TYPE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.support_timestamp")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. "
|
||||
+ "Disabled by default for backward compatibility.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_TABLE_PROPERTIES = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.table_properties")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Additional properties to store with table.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.serde_properties")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Serde properties to hive table.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.sync_as_datasource")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<Integer> HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.schema_string_length_thresh")
|
||||
.defaultValue(4000)
|
||||
.withDocumentation("");
|
||||
|
||||
// Create table as managed table
|
||||
public static final ConfigProperty<Boolean> HIVE_CREATE_MANAGED_TABLE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.create_managed_table")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Whether to sync the table as managed table.");
|
||||
|
||||
public static final ConfigProperty<Integer> HIVE_BATCH_SYNC_PARTITION_NUM = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.batch_num")
|
||||
.defaultValue(1000)
|
||||
.withDocumentation("The number of partitions one batch when synchronous partitions to hive.");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SYNC_MODE = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.mode")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.");
|
||||
|
||||
public static final ConfigProperty<Boolean> HIVE_SYNC_BUCKET_SYNC = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.bucket_sync")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Whether sync hive metastore bucket specification when using bucket index."
|
||||
+ "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SYNC_BUCKET_SYNC_SPEC = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.bucket_sync_spec")
|
||||
.defaultValue("")
|
||||
.withDocumentation("The hive metastore bucket specification when using bucket index."
|
||||
+ "The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_SYNC_COMMENT = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.sync_comment")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Whether to sync the table column comments while syncing the table.");
|
||||
|
||||
public HiveSyncConfig() {
|
||||
this(new TypedProperties());
|
||||
}
|
||||
|
||||
public HiveSyncConfig(TypedProperties props) {
|
||||
super(props);
|
||||
this.hiveUser = getStringOrDefault(HIVE_USER);
|
||||
this.hivePass = getStringOrDefault(HIVE_PASS);
|
||||
this.jdbcUrl = getStringOrDefault(HIVE_URL);
|
||||
this.usePreApacheInputFormat = getBooleanOrDefault(HIVE_USE_PRE_APACHE_INPUT_FORMAT);
|
||||
this.useJdbc = getBooleanOrDefault(HIVE_USE_JDBC);
|
||||
this.metastoreUris = getStringOrDefault(METASTORE_URIS);
|
||||
this.syncMode = getString(HIVE_SYNC_MODE);
|
||||
this.autoCreateDatabase = getBooleanOrDefault(HIVE_AUTO_CREATE_DATABASE);
|
||||
this.ignoreExceptions = getBooleanOrDefault(HIVE_IGNORE_EXCEPTIONS);
|
||||
this.skipROSuffix = getBooleanOrDefault(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE);
|
||||
this.tableProperties = getString(HIVE_TABLE_PROPERTIES);
|
||||
this.serdeProperties = getString(HIVE_TABLE_SERDE_PROPERTIES);
|
||||
this.supportTimestamp = getBooleanOrDefault(HIVE_SUPPORT_TIMESTAMP_TYPE);
|
||||
this.batchSyncNum = getIntOrDefault(HIVE_BATCH_SYNC_PARTITION_NUM);
|
||||
this.syncAsSparkDataSourceTable = getBooleanOrDefault(HIVE_SYNC_AS_DATA_SOURCE_TABLE);
|
||||
this.sparkSchemaLengthThreshold = getIntOrDefault(HIVE_SYNC_SCHEMA_STRING_LENGTH_THRESHOLD);
|
||||
this.createManagedTable = getBooleanOrDefault(HIVE_CREATE_MANAGED_TABLE);
|
||||
this.bucketSpec = getStringOrDefault(HIVE_SYNC_BUCKET_SYNC_SPEC);
|
||||
this.syncComment = getBooleanOrDefault(HIVE_SYNC_COMMENT);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -197,6 +275,7 @@ public class HiveSyncConfig implements Serializable {
|
||||
+ ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold
|
||||
+ ", withOperationField=" + withOperationField
|
||||
+ ", isConditionalSync=" + isConditionalSync
|
||||
+ ", sparkVersion=" + sparkVersion
|
||||
+ ", syncComment=" + syncComment
|
||||
+ '}';
|
||||
}
|
||||
|
||||
@@ -25,6 +25,8 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.hadoop.hive.metastore.api.FieldSchema;
|
||||
import org.apache.hadoop.hive.metastore.api.Partition;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
@@ -70,40 +72,53 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
public static final String SUFFIX_SNAPSHOT_TABLE = "_rt";
|
||||
public static final String SUFFIX_READ_OPTIMIZED_TABLE = "_ro";
|
||||
|
||||
protected final HiveSyncConfig cfg;
|
||||
protected final HiveSyncConfig hiveSyncConfig;
|
||||
protected HoodieHiveClient hoodieHiveClient = null;
|
||||
protected String snapshotTableName = null;
|
||||
protected Option<String> roTableName = null;
|
||||
|
||||
public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) {
|
||||
super(configuration.getAllProperties(), fs);
|
||||
public HiveSyncTool(TypedProperties props, Configuration conf, FileSystem fs) {
|
||||
super(props, conf, fs);
|
||||
this.hiveSyncConfig = new HiveSyncConfig(props);
|
||||
init(hiveSyncConfig, new HiveConf(conf, HiveConf.class));
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public HiveSyncTool(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf, FileSystem fs) {
|
||||
super(hiveSyncConfig.getProps(), hiveConf, fs);
|
||||
this.hiveSyncConfig = hiveSyncConfig;
|
||||
init(hiveSyncConfig, hiveConf);
|
||||
}
|
||||
|
||||
private void init(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) {
|
||||
try {
|
||||
this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs);
|
||||
if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) {
|
||||
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris);
|
||||
}
|
||||
this.hoodieHiveClient = new HoodieHiveClient(hiveSyncConfig, hiveConf, fs);
|
||||
} catch (RuntimeException e) {
|
||||
if (cfg.ignoreExceptions) {
|
||||
if (hiveSyncConfig.ignoreExceptions) {
|
||||
LOG.error("Got runtime exception when hive syncing, but continuing as ignoreExceptions config is set ", e);
|
||||
} else {
|
||||
throw new HoodieHiveSyncException("Got runtime exception when hive syncing", e);
|
||||
}
|
||||
}
|
||||
|
||||
this.cfg = cfg;
|
||||
// Set partitionFields to empty, when the NonPartitionedExtractor is used
|
||||
if (NonPartitionedExtractor.class.getName().equals(cfg.partitionValueExtractorClass)) {
|
||||
if (NonPartitionedExtractor.class.getName().equals(hiveSyncConfig.partitionValueExtractorClass)) {
|
||||
LOG.warn("Set partitionFields to empty, since the NonPartitionedExtractor is used");
|
||||
cfg.partitionFields = new ArrayList<>();
|
||||
hiveSyncConfig.partitionFields = new ArrayList<>();
|
||||
}
|
||||
if (hoodieHiveClient != null) {
|
||||
switch (hoodieHiveClient.getTableType()) {
|
||||
case COPY_ON_WRITE:
|
||||
this.snapshotTableName = cfg.tableName;
|
||||
this.snapshotTableName = hiveSyncConfig.tableName;
|
||||
this.roTableName = Option.empty();
|
||||
break;
|
||||
case MERGE_ON_READ:
|
||||
this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE;
|
||||
this.roTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) :
|
||||
Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
this.snapshotTableName = hiveSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE;
|
||||
this.roTableName = hiveSyncConfig.skipROSuffix ? Option.of(hiveSyncConfig.tableName) :
|
||||
Option.of(hiveSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
break;
|
||||
default:
|
||||
LOG.error("Unknown table type " + hoodieHiveClient.getTableType());
|
||||
@@ -116,10 +131,13 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
public void syncHoodieTable() {
|
||||
try {
|
||||
if (hoodieHiveClient != null) {
|
||||
LOG.info("Syncing target hoodie table with hive table(" + hiveSyncConfig.tableName + "). Hive metastore URL :"
|
||||
+ hiveSyncConfig.jdbcUrl + ", basePath :" + hiveSyncConfig.basePath);
|
||||
|
||||
doSync();
|
||||
}
|
||||
} catch (RuntimeException re) {
|
||||
throw new HoodieException("Got runtime exception when hive syncing " + cfg.tableName, re);
|
||||
throw new HoodieException("Got runtime exception when hive syncing " + hiveSyncConfig.tableName, re);
|
||||
} finally {
|
||||
if (hoodieHiveClient != null) {
|
||||
hoodieHiveClient.close();
|
||||
@@ -150,18 +168,19 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
+ " of type " + hoodieHiveClient.getTableType());
|
||||
|
||||
// check if the database exists else create it
|
||||
if (cfg.autoCreateDatabase) {
|
||||
if (hiveSyncConfig.autoCreateDatabase) {
|
||||
try {
|
||||
if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) {
|
||||
hoodieHiveClient.createDatabase(cfg.databaseName);
|
||||
if (!hoodieHiveClient.doesDataBaseExist(hiveSyncConfig.databaseName)) {
|
||||
hoodieHiveClient.createDatabase(hiveSyncConfig.databaseName);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
// this is harmless since table creation will fail anyways, creation of DB is needed for in-memory testing
|
||||
LOG.warn("Unable to create database", e);
|
||||
}
|
||||
} else {
|
||||
if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) {
|
||||
throw new HoodieHiveSyncException("hive database does not exist " + cfg.databaseName);
|
||||
if (!hoodieHiveClient.doesDataBaseExist(hiveSyncConfig.databaseName)) {
|
||||
LOG.error("Hive database does not exist " + hiveSyncConfig.databaseName);
|
||||
throw new HoodieHiveSyncException("hive database does not exist " + hiveSyncConfig.databaseName);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -181,7 +200,7 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
if (hoodieHiveClient.isBootstrap()
|
||||
&& hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ
|
||||
&& !readAsOptimized) {
|
||||
cfg.syncAsSparkDataSourceTable = false;
|
||||
hiveSyncConfig.syncAsSparkDataSourceTable = false;
|
||||
}
|
||||
|
||||
// Sync schema if needed
|
||||
@@ -200,7 +219,7 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
// Sync the partitions if needed
|
||||
boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition);
|
||||
boolean meetSyncConditions = schemaChanged || partitionsChanged;
|
||||
if (!cfg.isConditionalSync || meetSyncConditions) {
|
||||
if (!hiveSyncConfig.isConditionalSync || meetSyncConditions) {
|
||||
hoodieHiveClient.updateLastCommitTimeSynced(tableName);
|
||||
}
|
||||
LOG.info("Sync complete for " + tableName);
|
||||
@@ -216,10 +235,10 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
private boolean syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat,
|
||||
boolean readAsOptimized, MessageType schema) {
|
||||
// Append spark table properties & serde properties
|
||||
Map<String, String> tableProperties = ConfigUtils.toMap(cfg.tableProperties);
|
||||
Map<String, String> serdeProperties = ConfigUtils.toMap(cfg.serdeProperties);
|
||||
if (cfg.syncAsSparkDataSourceTable) {
|
||||
Map<String, String> sparkTableProperties = getSparkTableProperties(cfg.sparkSchemaLengthThreshold, schema);
|
||||
Map<String, String> tableProperties = ConfigUtils.toMap(hiveSyncConfig.tableProperties);
|
||||
Map<String, String> serdeProperties = ConfigUtils.toMap(hiveSyncConfig.serdeProperties);
|
||||
if (hiveSyncConfig.syncAsSparkDataSourceTable) {
|
||||
Map<String, String> sparkTableProperties = getSparkTableProperties(hiveSyncConfig.sparkSchemaLengthThreshold, schema);
|
||||
Map<String, String> sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized);
|
||||
tableProperties.putAll(sparkTableProperties);
|
||||
serdeProperties.putAll(sparkSerdeProperties);
|
||||
@@ -228,10 +247,10 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
// Check and sync schema
|
||||
if (!tableExists) {
|
||||
LOG.info("Hive table " + tableName + " is not found. Creating it");
|
||||
HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(cfg.baseFileFormat.toUpperCase());
|
||||
HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(hiveSyncConfig.baseFileFormat.toUpperCase());
|
||||
String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat);
|
||||
|
||||
if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && cfg.usePreApacheInputFormat) {
|
||||
if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && hiveSyncConfig.usePreApacheInputFormat) {
|
||||
// Parquet input format had an InputFormat class visible under the old naming scheme.
|
||||
inputFormatClassName = useRealTimeInputFormat
|
||||
? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName()
|
||||
@@ -250,12 +269,12 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
} else {
|
||||
// Check if the table schema has evolved
|
||||
Map<String, String> tableSchema = hoodieHiveClient.getTableSchema(tableName);
|
||||
SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields, cfg.supportTimestamp);
|
||||
SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, hiveSyncConfig.partitionFields, hiveSyncConfig.supportTimestamp);
|
||||
if (!schemaDiff.isEmpty()) {
|
||||
LOG.info("Schema difference found for " + tableName);
|
||||
hoodieHiveClient.updateTableDefinition(tableName, schema);
|
||||
// Sync the table properties if the schema has changed
|
||||
if (cfg.tableProperties != null || cfg.syncAsSparkDataSourceTable) {
|
||||
if (hiveSyncConfig.tableProperties != null || hiveSyncConfig.syncAsSparkDataSourceTable) {
|
||||
hoodieHiveClient.updateTableProperties(tableName, tableProperties);
|
||||
LOG.info("Sync table properties for " + tableName + ", table properties is: " + tableProperties);
|
||||
}
|
||||
@@ -265,7 +284,7 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
}
|
||||
}
|
||||
|
||||
if (cfg.syncComment) {
|
||||
if (hiveSyncConfig.syncComment) {
|
||||
Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields();
|
||||
Map<String, String> newComments = avroSchemaWithoutMetadataFields.getFields()
|
||||
.stream().collect(Collectors.toMap(Schema.Field::name, field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc()));
|
||||
@@ -290,7 +309,7 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
// The following code refers to the spark code in
|
||||
// https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala
|
||||
GroupType originGroupType = schema.asGroupType();
|
||||
List<String> partitionNames = cfg.partitionFields;
|
||||
List<String> partitionNames = hiveSyncConfig.partitionFields;
|
||||
List<Type> partitionCols = new ArrayList<>();
|
||||
List<Type> dataCols = new ArrayList<>();
|
||||
Map<String, Type> column2Field = new HashMap<>();
|
||||
@@ -319,8 +338,8 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
|
||||
Map<String, String> sparkProperties = new HashMap<>();
|
||||
sparkProperties.put("spark.sql.sources.provider", "hudi");
|
||||
if (!StringUtils.isNullOrEmpty(cfg.sparkVersion)) {
|
||||
sparkProperties.put("spark.sql.create.version", cfg.sparkVersion);
|
||||
if (!StringUtils.isNullOrEmpty(hiveSyncConfig.sparkVersion)) {
|
||||
sparkProperties.put("spark.sql.create.version", hiveSyncConfig.sparkVersion);
|
||||
}
|
||||
// Split the schema string to multi-parts according the schemaLengthThreshold size.
|
||||
String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType);
|
||||
@@ -344,7 +363,7 @@ public class HiveSyncTool extends AbstractSyncTool {
|
||||
|
||||
private Map<String, String> getSparkSerdeProperties(boolean readAsOptimized) {
|
||||
Map<String, String> sparkSerdeProperties = new HashMap<>();
|
||||
sparkSerdeProperties.put("path", cfg.basePath);
|
||||
sparkSerdeProperties.put("path", hiveSyncConfig.basePath);
|
||||
sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized));
|
||||
return sparkSerdeProperties;
|
||||
}
|
||||
|
||||
@@ -18,15 +18,24 @@
|
||||
|
||||
package org.apache.hudi.hive.replication;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
public class GlobalHiveSyncConfig extends HiveSyncConfig {
|
||||
@Parameter(names = {"--replicated-timestamp"}, description = "Add globally replicated timestamp to enable consistent reads across clusters")
|
||||
public String globallyReplicatedTimeStamp;
|
||||
|
||||
public GlobalHiveSyncConfig() {
|
||||
}
|
||||
|
||||
public GlobalHiveSyncConfig(TypedProperties props) {
|
||||
super(props);
|
||||
}
|
||||
|
||||
public static GlobalHiveSyncConfig copy(GlobalHiveSyncConfig cfg) {
|
||||
GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig();
|
||||
GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(cfg.getProps());
|
||||
newConfig.basePath = cfg.basePath;
|
||||
newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning;
|
||||
newConfig.databaseName = cfg.databaseName;
|
||||
|
||||
@@ -48,9 +48,9 @@ public class GlobalHiveSyncTool extends HiveSyncTool {
|
||||
@Override
|
||||
protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) {
|
||||
super.syncHoodieTable(tableName, useRealtimeInputFormat, readAsOptimized);
|
||||
if (((GlobalHiveSyncConfig)cfg).globallyReplicatedTimeStamp != null) {
|
||||
if (((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp != null) {
|
||||
hoodieHiveClient.updateLastReplicatedTimeStamp(tableName,
|
||||
((GlobalHiveSyncConfig) cfg).globallyReplicatedTimeStamp);
|
||||
((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp);
|
||||
}
|
||||
LOG.info("Sync complete for " + tableName);
|
||||
}
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -22,6 +22,7 @@ import org.apache.hudi.avro.HoodieAvroWriteSupport;
|
||||
import org.apache.hudi.common.bloom.BloomFilter;
|
||||
import org.apache.hudi.common.bloom.BloomFilterFactory;
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.HoodieAvroPayload;
|
||||
import org.apache.hudi.common.model.HoodieBaseFile;
|
||||
@@ -75,7 +76,6 @@ import java.time.ZonedDateTime;
|
||||
import java.time.format.DateTimeFormatter;
|
||||
import java.time.temporal.ChronoUnit;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
@@ -89,16 +89,21 @@ import static org.junit.jupiter.api.Assertions.fail;
|
||||
@SuppressWarnings("SameParameterValue")
|
||||
public class HiveTestUtil {
|
||||
|
||||
public static final String DB_NAME = "testdb";
|
||||
public static final String TABLE_NAME = "test1";
|
||||
public static String basePath;
|
||||
public static TypedProperties hiveSyncProps;
|
||||
public static HiveTestService hiveTestService;
|
||||
public static FileSystem fileSystem;
|
||||
public static QueryBasedDDLExecutor ddlExecutor;
|
||||
|
||||
private static ZooKeeperServer zkServer;
|
||||
private static HiveServer2 hiveServer;
|
||||
public static HiveTestService hiveTestService;
|
||||
private static ZookeeperTestService zkService;
|
||||
private static Configuration configuration;
|
||||
public static HiveSyncConfig hiveSyncConfig;
|
||||
private static HiveSyncConfig hiveSyncConfig;
|
||||
private static DateTimeFormatter dtfOut;
|
||||
public static FileSystem fileSystem;
|
||||
private static Set<String> createdTablesSet = new HashSet<>();
|
||||
public static QueryBasedDDLExecutor ddlExecutor;
|
||||
|
||||
public static void setUp() throws IOException, InterruptedException, HiveException, MetaException {
|
||||
configuration = new Configuration();
|
||||
@@ -112,16 +117,21 @@ public class HiveTestUtil {
|
||||
}
|
||||
fileSystem = FileSystem.get(configuration);
|
||||
|
||||
hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.jdbcUrl = hiveTestService.getJdbcHive2Url();
|
||||
hiveSyncConfig.hiveUser = "";
|
||||
hiveSyncConfig.hivePass = "";
|
||||
hiveSyncConfig.databaseName = "testdb";
|
||||
hiveSyncConfig.tableName = "test1";
|
||||
hiveSyncConfig.basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString();
|
||||
hiveSyncConfig.assumeDatePartitioning = true;
|
||||
hiveSyncConfig.usePreApacheInputFormat = false;
|
||||
hiveSyncConfig.partitionFields = Collections.singletonList("datestr");
|
||||
basePath = Files.createTempDirectory("hivesynctest" + Instant.now().toEpochMilli()).toUri().toString();
|
||||
|
||||
hiveSyncProps = new TypedProperties();
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveTestService.getJdbcHive2Url());
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USER.key(), "");
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_PASS.key(), "");
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), DB_NAME);
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), TABLE_NAME);
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_BASE_PATH.key(), basePath);
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "true");
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false");
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr");
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_BATCH_SYNC_PARTITION_NUM.key(), "3");
|
||||
|
||||
hiveSyncConfig = new HiveSyncConfig(hiveSyncProps);
|
||||
|
||||
dtfOut = DateTimeFormatter.ofPattern("yyyy/MM/dd");
|
||||
ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig, fileSystem, getHiveConf());
|
||||
@@ -138,18 +148,18 @@ public class HiveTestUtil {
|
||||
}
|
||||
|
||||
public static void clear() throws IOException, HiveException, MetaException {
|
||||
fileSystem.delete(new Path(hiveSyncConfig.basePath), true);
|
||||
fileSystem.delete(new Path(basePath), true);
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.COPY_ON_WRITE)
|
||||
.setTableName(hiveSyncConfig.tableName)
|
||||
.setTableName(TABLE_NAME)
|
||||
.setPayloadClass(HoodieAvroPayload.class)
|
||||
.initTable(configuration, hiveSyncConfig.basePath);
|
||||
.initTable(configuration, basePath);
|
||||
|
||||
for (String tableName : createdTablesSet) {
|
||||
ddlExecutor.runSQL("drop table if exists " + tableName);
|
||||
}
|
||||
createdTablesSet.clear();
|
||||
ddlExecutor.runSQL("drop database if exists " + hiveSyncConfig.databaseName + " cascade");
|
||||
ddlExecutor.runSQL("drop database if exists " + DB_NAME + " cascade");
|
||||
}
|
||||
|
||||
public static HiveConf getHiveConf() {
|
||||
@@ -189,7 +199,7 @@ public class HiveTestUtil {
|
||||
|
||||
public static void createCOWTable(String instantTime, int numberOfPartitions, boolean useSchemaFromCommitMetadata)
|
||||
throws IOException, URISyntaxException {
|
||||
createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, hiveSyncConfig.basePath, hiveSyncConfig.databaseName, hiveSyncConfig.tableName);
|
||||
createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, basePath, DB_NAME, TABLE_NAME);
|
||||
}
|
||||
|
||||
public static void createReplaceCommit(String instantTime, String partitions, WriteOperationType type, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata)
|
||||
@@ -205,13 +215,13 @@ public class HiveTestUtil {
|
||||
|
||||
public static void createCOWTableWithSchema(String instantTime, String schemaFileName)
|
||||
throws IOException, URISyntaxException {
|
||||
Path path = new Path(hiveSyncConfig.basePath);
|
||||
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
|
||||
Path path = new Path(basePath);
|
||||
FileIOUtils.deleteDirectory(new File(basePath));
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.COPY_ON_WRITE)
|
||||
.setTableName(hiveSyncConfig.tableName)
|
||||
.setTableName(TABLE_NAME)
|
||||
.setPayloadClass(HoodieAvroPayload.class)
|
||||
.initTable(configuration, hiveSyncConfig.basePath);
|
||||
.initTable(configuration, basePath);
|
||||
|
||||
boolean result = fileSystem.mkdirs(path);
|
||||
checkResult(result);
|
||||
@@ -219,7 +229,7 @@ public class HiveTestUtil {
|
||||
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
String partitionPath = dateTime.format(dtfOut);
|
||||
Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath);
|
||||
Path partPath = new Path(basePath + "/" + partitionPath);
|
||||
fileSystem.makeQualified(partPath);
|
||||
fileSystem.mkdirs(partPath);
|
||||
List<HoodieWriteStat> writeStats = new ArrayList<>();
|
||||
@@ -233,30 +243,30 @@ public class HiveTestUtil {
|
||||
writeStats.add(writeStat);
|
||||
writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s));
|
||||
commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schema.toString());
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath);
|
||||
createdTablesSet.add(DB_NAME + "." + TABLE_NAME);
|
||||
createCommitFile(commitMetadata, instantTime, basePath);
|
||||
}
|
||||
|
||||
public static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions,
|
||||
boolean createDeltaCommit, boolean useSchemaFromCommitMetadata)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
Path path = new Path(hiveSyncConfig.basePath);
|
||||
FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath));
|
||||
Path path = new Path(basePath);
|
||||
FileIOUtils.deleteDirectory(new File(basePath));
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.MERGE_ON_READ)
|
||||
.setTableName(hiveSyncConfig.tableName)
|
||||
.setTableName(TABLE_NAME)
|
||||
.setPayloadClass(HoodieAvroPayload.class)
|
||||
.initTable(configuration, hiveSyncConfig.basePath);
|
||||
.initTable(configuration, basePath);
|
||||
|
||||
boolean result = fileSystem.mkdirs(path);
|
||||
checkResult(result);
|
||||
ZonedDateTime dateTime = ZonedDateTime.now();
|
||||
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true,
|
||||
useSchemaFromCommitMetadata, dateTime, commitTime, hiveSyncConfig.basePath);
|
||||
useSchemaFromCommitMetadata, dateTime, commitTime, basePath);
|
||||
createdTablesSet
|
||||
.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
createdTablesSet
|
||||
.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.getPartitionToWriteStats()
|
||||
.forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l)));
|
||||
@@ -274,26 +284,26 @@ public class HiveTestUtil {
|
||||
public static void addCOWPartitions(int numberOfPartitions, boolean isParquetSchemaSimple,
|
||||
boolean useSchemaFromCommitMetadata, ZonedDateTime startFrom, String instantTime) throws IOException, URISyntaxException {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime, hiveSyncConfig.basePath);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath);
|
||||
createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime, basePath);
|
||||
createdTablesSet.add(DB_NAME + "." + TABLE_NAME);
|
||||
createCommitFile(commitMetadata, instantTime, basePath);
|
||||
}
|
||||
|
||||
public static void addCOWPartition(String partitionPath, boolean isParquetSchemaSimple,
|
||||
boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException {
|
||||
HoodieCommitMetadata commitMetadata =
|
||||
createPartition(partitionPath, isParquetSchemaSimple, useSchemaFromCommitMetadata, instantTime);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName);
|
||||
createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath);
|
||||
createdTablesSet.add(DB_NAME + "." + TABLE_NAME);
|
||||
createCommitFile(commitMetadata, instantTime, basePath);
|
||||
}
|
||||
|
||||
public static void addMORPartitions(int numberOfPartitions, boolean isParquetSchemaSimple, boolean isLogSchemaSimple,
|
||||
boolean useSchemaFromCommitMetadata, ZonedDateTime startFrom, String instantTime, String deltaCommitTime)
|
||||
throws IOException, URISyntaxException, InterruptedException {
|
||||
HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, isParquetSchemaSimple,
|
||||
useSchemaFromCommitMetadata, startFrom, instantTime, hiveSyncConfig.basePath);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
useSchemaFromCommitMetadata, startFrom, instantTime, basePath);
|
||||
createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE);
|
||||
createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE);
|
||||
HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata();
|
||||
commitMetadata.getPartitionToWriteStats()
|
||||
.forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l)));
|
||||
@@ -346,7 +356,7 @@ public class HiveTestUtil {
|
||||
private static HoodieCommitMetadata createPartition(String partitionPath, boolean isParquetSchemaSimple,
|
||||
boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException {
|
||||
HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata();
|
||||
Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath);
|
||||
Path partPath = new Path(basePath + "/" + partitionPath);
|
||||
fileSystem.makeQualified(partPath);
|
||||
fileSystem.mkdirs(partPath);
|
||||
List<HoodieWriteStat> writeStats = createTestData(partPath, isParquetSchemaSimple, instantTime);
|
||||
@@ -471,7 +481,7 @@ public class HiveTestUtil {
|
||||
|
||||
public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException {
|
||||
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeReplaceFileName(instantTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
@@ -480,13 +490,13 @@ public class HiveTestUtil {
|
||||
|
||||
public static void createCommitFileWithSchema(HoodieCommitMetadata commitMetadata, String instantTime, boolean isSimpleSchema) throws IOException {
|
||||
addSchemaToCommitMetadata(commitMetadata, isSimpleSchema, true);
|
||||
createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath);
|
||||
createCommitFile(commitMetadata, instantTime, basePath);
|
||||
}
|
||||
|
||||
private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, String instantTime)
|
||||
throws IOException {
|
||||
byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeCommitFileName(instantTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
@@ -496,7 +506,7 @@ public class HiveTestUtil {
|
||||
private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime)
|
||||
throws IOException {
|
||||
byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8);
|
||||
Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/"
|
||||
+ HoodieTimeline.makeDeltaFileName(deltaCommitTime));
|
||||
FSDataOutputStream fsout = fileSystem.create(fullPath, true);
|
||||
fsout.write(bytes);
|
||||
|
||||
@@ -45,6 +45,65 @@
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.beust</groupId>
|
||||
<artifactId>jcommander</artifactId>
|
||||
</dependency>
|
||||
|
||||
<!-- Required for Testing -->
|
||||
<dependency>
|
||||
<groupId>org.apache.logging.log4j</groupId>
|
||||
<artifactId>log4j-core</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.vintage</groupId>
|
||||
<artifactId>junit-vintage-engine</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.jupiter</groupId>
|
||||
<artifactId>junit-jupiter-params</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-junit-jupiter</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-runner</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-suite-api</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.junit.platform</groupId>
|
||||
<artifactId>junit-platform-commons</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
|
||||
<build>
|
||||
|
||||
@@ -17,17 +17,31 @@
|
||||
|
||||
package org.apache.hudi.sync.common;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Base class to sync Hudi meta data with Metastores to make
|
||||
* Hudi table queryable through external systems.
|
||||
*/
|
||||
public abstract class AbstractSyncTool {
|
||||
protected Properties props;
|
||||
protected FileSystem fileSystem;
|
||||
protected final Configuration conf;
|
||||
protected final FileSystem fs;
|
||||
protected TypedProperties props;
|
||||
|
||||
public AbstractSyncTool(Properties props, FileSystem fileSystem) {
|
||||
public AbstractSyncTool(TypedProperties props, Configuration conf, FileSystem fs) {
|
||||
this.props = props;
|
||||
this.fileSystem = fileSystem;
|
||||
this.conf = conf;
|
||||
this.fs = fs;
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
public AbstractSyncTool(Properties props, FileSystem fileSystem) {
|
||||
this(new TypedProperties(props), fileSystem.getConf(), fileSystem);
|
||||
}
|
||||
|
||||
public abstract void syncHoodieTable();
|
||||
|
||||
@@ -0,0 +1,188 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sync.common;
|
||||
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.function.Function;
|
||||
|
||||
/**
|
||||
* Configs needed to sync data into external meta stores, catalogs, etc.
|
||||
*/
|
||||
public class HoodieSyncConfig extends HoodieConfig {
|
||||
|
||||
@Parameter(names = {"--database"}, description = "name of the target database in meta store", required = true)
|
||||
public String databaseName;
|
||||
|
||||
@Parameter(names = {"--table"}, description = "name of the target table in meta store", required = true)
|
||||
public String tableName;
|
||||
|
||||
@Parameter(names = {"--base-path"}, description = "Base path of the hoodie table to sync", required = true)
|
||||
public String basePath;
|
||||
|
||||
@Parameter(names = {"--base-file-format"}, description = "Format of the base files (PARQUET (or) HFILE)")
|
||||
public String baseFileFormat;
|
||||
|
||||
@Parameter(names = "--partitioned-by", description = "Fields in the schema partitioned by")
|
||||
public List<String> partitionFields;
|
||||
|
||||
@Parameter(names = "--partition-value-extractor", description = "Class which implements PartitionValueExtractor "
|
||||
+ "to extract the partition values from HDFS path")
|
||||
public String partitionValueExtractorClass;
|
||||
|
||||
@Parameter(names = {"--assume-date-partitioning"}, description = "Assume standard yyyy/mm/dd partitioning, this"
|
||||
+ " exists to support backward compatibility. If you use hoodie 0.3.x, do not set this parameter")
|
||||
public Boolean assumeDatePartitioning;
|
||||
|
||||
@Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing")
|
||||
public Boolean decodePartition;
|
||||
|
||||
@Parameter(names = {"--use-file-listing-from-metadata"}, description = "Fetch file listing from Hudi's metadata")
|
||||
public Boolean useFileListingFromMetadata;
|
||||
|
||||
@Parameter(names = {"--conditional-sync"}, description = "If true, only sync on conditions like schema change or partition change.")
|
||||
public Boolean isConditionalSync;
|
||||
|
||||
@Parameter(names = {"--spark-version"}, description = "The spark version")
|
||||
public String sparkVersion;
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_BASE_PATH = ConfigProperty
|
||||
.key("hoodie.datasource.meta.sync.base.path")
|
||||
.defaultValue("")
|
||||
.withDocumentation("Base path of the hoodie table to sync");
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_ENABLED = ConfigProperty
|
||||
.key("hoodie.datasource.meta.sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Enable Syncing the Hudi Table with an external meta store or data catalog.");
|
||||
|
||||
// ToDo change the prefix of the following configs from hive_sync to meta_sync
|
||||
public static final ConfigProperty<String> META_SYNC_DATABASE_NAME = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.database")
|
||||
.defaultValue("default")
|
||||
.withDocumentation("The name of the destination database that we should sync the hudi table to.");
|
||||
|
||||
// If the table name for the metastore destination is not provided, pick it up from write or table configs.
|
||||
public static final Function<HoodieConfig, Option<String>> TABLE_NAME_INFERENCE_FUNCTION = cfg -> {
|
||||
if (cfg.contains(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY)) {
|
||||
return Option.of(cfg.getString(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY));
|
||||
} else if (cfg.contains(HoodieTableConfig.HOODIE_TABLE_NAME_KEY)) {
|
||||
return Option.of(cfg.getString(HoodieTableConfig.HOODIE_TABLE_NAME_KEY));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
};
|
||||
public static final ConfigProperty<String> META_SYNC_TABLE_NAME = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.table")
|
||||
.defaultValue("unknown")
|
||||
.withInferFunction(TABLE_NAME_INFERENCE_FUNCTION)
|
||||
.withDocumentation("The name of the destination table that we should sync the hudi table to.");
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_BASE_FILE_FORMAT = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.base_file_format")
|
||||
.defaultValue("PARQUET")
|
||||
.withDocumentation("Base file format for the sync.");
|
||||
|
||||
// If partition fields are not explicitly provided, obtain from the KeyGeneration Configs
|
||||
public static final Function<HoodieConfig, Option<String>> PARTITION_FIELDS_INFERENCE_FUNCTION = cfg -> {
|
||||
if (cfg.contains(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)) {
|
||||
return Option.of(cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
};
|
||||
public static final ConfigProperty<String> META_SYNC_PARTITION_FIELDS = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.partition_fields")
|
||||
.defaultValue("")
|
||||
.withInferFunction(PARTITION_FIELDS_INFERENCE_FUNCTION)
|
||||
.withDocumentation("Field in the table to use for determining hive partition columns.");
|
||||
|
||||
// If partition value extraction class is not explicitly provided, configure based on the partition fields.
|
||||
public static final Function<HoodieConfig, Option<String>> PARTITION_EXTRACTOR_CLASS_FUNCTION = cfg -> {
|
||||
if (!cfg.contains(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)) {
|
||||
return Option.of("org.apache.hudi.hive.NonPartitionedExtractor");
|
||||
} else {
|
||||
int numOfPartFields = cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME).split(",").length;
|
||||
if (numOfPartFields == 1
|
||||
&& cfg.contains(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE)
|
||||
&& cfg.getString(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE).equals("true")) {
|
||||
return Option.of("org.apache.hudi.hive.HiveStylePartitionValueExtractor");
|
||||
} else {
|
||||
return Option.of("org.apache.hudi.hive.MultiPartKeysValueExtractor");
|
||||
}
|
||||
}
|
||||
};
|
||||
public static final ConfigProperty<String> META_SYNC_PARTITION_EXTRACTOR_CLASS = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.partition_extractor_class")
|
||||
.defaultValue("org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor")
|
||||
.withInferFunction(PARTITION_EXTRACTOR_CLASS_FUNCTION)
|
||||
.withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, "
|
||||
+ "default 'SlashEncodedDayPartitionValueExtractor'.");
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_ASSUME_DATE_PARTITION = ConfigProperty
|
||||
.key("hoodie.datasource.hive_sync.assume_date_partitioning")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Assume partitioning is yyyy/mm/dd");
|
||||
|
||||
public static final ConfigProperty<Boolean> META_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty
|
||||
.key("hoodie.meta.sync.metadata_file_listing")
|
||||
.defaultValue(HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS)
|
||||
.withDocumentation("Enable the internal metadata table for file listing for syncing with metastores");
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_CONDITIONAL_SYNC = ConfigProperty
|
||||
.key("hoodie.datasource.meta_sync.condition.sync")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("If true, only sync on conditions like schema change or partition change.");
|
||||
|
||||
public static final ConfigProperty<String> META_SYNC_SPARK_VERSION = ConfigProperty
|
||||
.key("hoodie.meta_sync.spark.version")
|
||||
.defaultValue("")
|
||||
.withDocumentation("The spark version used when syncing with a metastore.");
|
||||
|
||||
public HoodieSyncConfig(TypedProperties props) {
|
||||
super(props);
|
||||
setDefaults();
|
||||
|
||||
this.basePath = getStringOrDefault(META_SYNC_BASE_PATH);
|
||||
this.databaseName = getStringOrDefault(META_SYNC_DATABASE_NAME);
|
||||
this.tableName = getStringOrDefault(META_SYNC_TABLE_NAME);
|
||||
this.baseFileFormat = getStringOrDefault(META_SYNC_BASE_FILE_FORMAT);
|
||||
this.partitionFields = props.getStringList(META_SYNC_PARTITION_FIELDS.key(), ",", Collections.emptyList());
|
||||
this.partitionValueExtractorClass = getStringOrDefault(META_SYNC_PARTITION_EXTRACTOR_CLASS);
|
||||
this.assumeDatePartitioning = getBooleanOrDefault(META_SYNC_ASSUME_DATE_PARTITION);
|
||||
this.decodePartition = getBooleanOrDefault(KeyGeneratorOptions.URL_ENCODE_PARTITIONING);
|
||||
this.useFileListingFromMetadata = getBooleanOrDefault(META_SYNC_USE_FILE_LISTING_FROM_METADATA);
|
||||
this.isConditionalSync = getBooleanOrDefault(META_SYNC_CONDITIONAL_SYNC);
|
||||
this.sparkVersion = getStringOrDefault(META_SYNC_SPARK_VERSION);
|
||||
}
|
||||
|
||||
protected void setDefaults() {
|
||||
this.setDefaultValue(META_SYNC_TABLE_NAME);
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,72 @@
|
||||
package org.apache.hudi.sync.common.util;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Helper class for syncing Hudi commit data with external metastores.
|
||||
*/
|
||||
public class SyncUtilHelpers {
|
||||
private static final Logger LOG = LogManager.getLogger(SyncUtilHelpers.class);
|
||||
|
||||
/**
|
||||
* Create an instance of an implementation of {@link AbstractSyncTool} that will sync all the relevant meta information
|
||||
* with an external metastore such as Hive etc. to ensure Hoodie tables can be queried or read via external systems.
|
||||
*
|
||||
* @param metaSyncFQCN The class that implements the sync of the metadata.
|
||||
* @param props property map.
|
||||
* @param hadoopConfig Hadoop confs.
|
||||
* @param fs Filesystem used.
|
||||
* @param targetBasePath The target base path that contains the hoodie table.
|
||||
* @param baseFileFormat The file format used by the hoodie table (defaults to PARQUET).
|
||||
*/
|
||||
public static void runHoodieMetaSync(String metaSyncFQCN,
|
||||
TypedProperties props,
|
||||
Configuration hadoopConfig,
|
||||
FileSystem fs,
|
||||
String targetBasePath,
|
||||
String baseFileFormat) {
|
||||
try {
|
||||
instantiateMetaSyncTool(metaSyncFQCN, props, hadoopConfig, fs, targetBasePath, baseFileFormat).syncHoodieTable();
|
||||
} catch (Throwable e) {
|
||||
throw new HoodieException("Could not sync using the meta sync class " + metaSyncFQCN, e);
|
||||
}
|
||||
}
|
||||
|
||||
static AbstractSyncTool instantiateMetaSyncTool(String metaSyncFQCN,
|
||||
TypedProperties props,
|
||||
Configuration hadoopConfig,
|
||||
FileSystem fs,
|
||||
String targetBasePath,
|
||||
String baseFileFormat) {
|
||||
TypedProperties properties = new TypedProperties();
|
||||
properties.putAll(props);
|
||||
properties.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), targetBasePath);
|
||||
properties.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), baseFileFormat);
|
||||
|
||||
if (ReflectionUtils.hasConstructor(metaSyncFQCN,
|
||||
new Class<?>[] {TypedProperties.class, Configuration.class, FileSystem.class})) {
|
||||
return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN,
|
||||
new Class<?>[] {TypedProperties.class, Configuration.class, FileSystem.class},
|
||||
properties, hadoopConfig, fs));
|
||||
} else {
|
||||
LOG.warn("Falling back to deprecated constructor for class: " + metaSyncFQCN);
|
||||
try {
|
||||
return ((AbstractSyncTool) ReflectionUtils.loadClass(metaSyncFQCN,
|
||||
new Class<?>[] {Properties.class, FileSystem.class}, properties, fs));
|
||||
} catch (Throwable t) {
|
||||
throw new HoodieException("Could not load meta sync class " + metaSyncFQCN, t);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,124 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.sync.common.util;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public class TestSyncUtilHelpers {
|
||||
private static final String BASE_PATH = "/tmp/test";
|
||||
private static final String BASE_FORMAT = "PARQUET";
|
||||
|
||||
private Configuration hadoopConf;
|
||||
private FileSystem fileSystem;
|
||||
|
||||
@BeforeEach
|
||||
public void setUp() throws IOException {
|
||||
fileSystem = FSUtils.getFs(BASE_PATH, new Configuration());
|
||||
hadoopConf = fileSystem.getConf();
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateValidSyncClass() {
|
||||
AbstractSyncTool metaSyncTool = SyncUtilHelpers.instantiateMetaSyncTool(
|
||||
ValidMetaSyncClass.class.getName(),
|
||||
new TypedProperties(),
|
||||
hadoopConf,
|
||||
fileSystem,
|
||||
BASE_PATH,
|
||||
BASE_FORMAT
|
||||
);
|
||||
assertTrue(metaSyncTool instanceof ValidMetaSyncClass);
|
||||
}
|
||||
|
||||
/**
|
||||
* Ensure it still works for the deprecated constructor of {@link AbstractSyncTool}
|
||||
* as we implemented the fallback.
|
||||
*/
|
||||
@Test
|
||||
public void testCreateDeprecatedSyncClass() {
|
||||
Properties properties = new Properties();
|
||||
AbstractSyncTool deprecatedMetaSyncClass = SyncUtilHelpers.instantiateMetaSyncTool(
|
||||
DeprecatedMetaSyncClass.class.getName(),
|
||||
new TypedProperties(properties),
|
||||
hadoopConf,
|
||||
fileSystem,
|
||||
BASE_PATH,
|
||||
BASE_FORMAT
|
||||
);
|
||||
assertTrue(deprecatedMetaSyncClass instanceof DeprecatedMetaSyncClass);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCreateInvalidSyncClass() {
|
||||
Exception exception = assertThrows(HoodieException.class, () -> {
|
||||
SyncUtilHelpers.instantiateMetaSyncTool(
|
||||
InvalidSyncClass.class.getName(),
|
||||
new TypedProperties(),
|
||||
hadoopConf,
|
||||
fileSystem,
|
||||
BASE_PATH,
|
||||
BASE_FORMAT
|
||||
);
|
||||
});
|
||||
|
||||
String expectedMessage = "Could not load meta sync class " + InvalidSyncClass.class.getName();
|
||||
assertTrue(exception.getMessage().contains(expectedMessage));
|
||||
|
||||
}
|
||||
|
||||
public static class ValidMetaSyncClass extends AbstractSyncTool {
|
||||
public ValidMetaSyncClass(TypedProperties props, Configuration conf, FileSystem fs) {
|
||||
super(props, conf, fs);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void syncHoodieTable() {
|
||||
throw new HoodieException("Method unimplemented as its a test class");
|
||||
}
|
||||
}
|
||||
|
||||
public static class DeprecatedMetaSyncClass extends AbstractSyncTool {
|
||||
public DeprecatedMetaSyncClass(Properties props, FileSystem fileSystem) {
|
||||
super(props, fileSystem);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void syncHoodieTable() {
|
||||
throw new HoodieException("Method unimplemented as its a test class");
|
||||
}
|
||||
}
|
||||
|
||||
public static class InvalidSyncClass {
|
||||
public InvalidSyncClass(Properties props) {
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,29 @@
|
||||
###
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
|
||||
# CONSOLE is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# CONSOLE uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=[%-5p] %d %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -0,0 +1,30 @@
|
||||
###
|
||||
# Licensed to the Apache Software Foundation (ASF) under one
|
||||
# or more contributor license agreements. See the NOTICE file
|
||||
# distributed with this work for additional information
|
||||
# regarding copyright ownership. The ASF licenses this file
|
||||
# to you under the Apache License, Version 2.0 (the
|
||||
# "License"); you may not use this file except in compliance
|
||||
# with the License. You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
###
|
||||
log4j.rootLogger=WARN, CONSOLE
|
||||
log4j.logger.org.apache=INFO
|
||||
log4j.logger.org.apache.hudi=DEBUG
|
||||
|
||||
# A1 is set to be a ConsoleAppender.
|
||||
log4j.appender.CONSOLE=org.apache.log4j.ConsoleAppender
|
||||
# A1 uses PatternLayout.
|
||||
log4j.appender.CONSOLE.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.CONSOLE.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n
|
||||
log4j.appender.CONSOLE.filter.a=org.apache.log4j.varia.LevelRangeFilter
|
||||
log4j.appender.CONSOLE.filter.a.AcceptOnMatch=true
|
||||
log4j.appender.CONSOLE.filter.a.LevelMin=WARN
|
||||
log4j.appender.CONSOLE.filter.a.LevelMax=FATAL
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import org.apache.hudi.DataSourceUtils;
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
@@ -34,13 +33,13 @@ import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.schema.SchemaProvider;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaSparkContext;
|
||||
@@ -161,12 +160,16 @@ public class BootstrapExecutor implements Serializable {
|
||||
*/
|
||||
private void syncHive() {
|
||||
if (cfg.enableHiveSync || cfg.enableMetaSync) {
|
||||
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
|
||||
HiveConf hiveConf = new HiveConf(fs.getConf(), HiveConf.class);
|
||||
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname,hiveSyncConfig.metastoreUris);
|
||||
LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString());
|
||||
LOG.info("Hive Sync Conf => " + hiveSyncConfig);
|
||||
new HiveSyncTool(hiveSyncConfig, new HiveConf(configuration, HiveConf.class), fs).syncHoodieTable();
|
||||
TypedProperties metaProps = new TypedProperties();
|
||||
metaProps.putAll(props);
|
||||
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), cfg.targetBasePath);
|
||||
metaProps.put(HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key(), cfg.baseFileFormat);
|
||||
if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
|
||||
metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
|
||||
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())));
|
||||
}
|
||||
|
||||
new HiveSyncTool(metaProps, configuration, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
@@ -195,4 +198,4 @@ public class BootstrapExecutor implements Serializable {
|
||||
public HoodieWriteConfig getBootstrapConfig() {
|
||||
return bootstrapConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -42,12 +42,12 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.util.CommitUtils;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.common.util.collection.Pair;
|
||||
import org.apache.hudi.config.HoodieClusteringConfig;
|
||||
import org.apache.hudi.config.HoodieCompactionConfig;
|
||||
import org.apache.hudi.config.HoodieIndexConfig;
|
||||
import org.apache.hudi.config.HoodiePayloadConfig;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
@@ -59,7 +59,7 @@ import org.apache.hudi.keygen.SimpleKeyGenerator;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
|
||||
import org.apache.hudi.metrics.HoodieMetrics;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
import org.apache.hudi.sync.common.util.SyncUtilHelpers;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback;
|
||||
import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig;
|
||||
@@ -81,7 +81,6 @@ import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
@@ -99,7 +98,6 @@ import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.function.Function;
|
||||
import java.util.stream.Collectors;
|
||||
@@ -692,44 +690,24 @@ public class DeltaSync implements Serializable {
|
||||
LOG.info("When set --enable-hive-sync will use HiveSyncTool for backward compatibility");
|
||||
}
|
||||
if (cfg.enableMetaSync) {
|
||||
FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
|
||||
|
||||
TypedProperties metaProps = new TypedProperties();
|
||||
metaProps.putAll(props);
|
||||
if (props.getBoolean(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.key(), HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC.defaultValue())) {
|
||||
metaProps.put(HiveSyncConfig.HIVE_SYNC_BUCKET_SYNC_SPEC.key(), HiveSyncConfig.getBucketSpec(props.getString(HoodieIndexConfig.BUCKET_INDEX_HASH_FIELD.key()),
|
||||
props.getInteger(HoodieIndexConfig.BUCKET_INDEX_NUM_BUCKETS.key())));
|
||||
}
|
||||
|
||||
for (String impl : syncClientToolClasses) {
|
||||
Timer.Context syncContext = metrics.getMetaSyncTimerContext();
|
||||
impl = impl.trim();
|
||||
switch (impl) {
|
||||
case "org.apache.hudi.hive.HiveSyncTool":
|
||||
syncHive();
|
||||
break;
|
||||
default:
|
||||
FileSystem fs = FSUtils.getFs(cfg.targetBasePath, jssc.hadoopConfiguration());
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(props);
|
||||
properties.put("basePath", cfg.targetBasePath);
|
||||
properties.put("baseFileFormat", cfg.baseFileFormat);
|
||||
AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[]{Properties.class, FileSystem.class}, properties, fs);
|
||||
syncTool.syncHoodieTable();
|
||||
}
|
||||
SyncUtilHelpers.runHoodieMetaSync(impl.trim(), metaProps, conf, fs, cfg.targetBasePath, cfg.baseFileFormat);
|
||||
long metaSyncTimeMs = syncContext != null ? syncContext.stop() : 0;
|
||||
metrics.updateDeltaStreamerMetaSyncMetrics(getSyncClassShortName(impl), metaSyncTimeMs);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public void syncHive() {
|
||||
HiveSyncConfig hiveSyncConfig = DataSourceUtils.buildHiveSyncConfig(props, cfg.targetBasePath, cfg.baseFileFormat);
|
||||
HiveConf hiveConf = new HiveConf(conf, HiveConf.class);
|
||||
if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) {
|
||||
hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris);
|
||||
}
|
||||
LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString());
|
||||
LOG.info("Hive Sync Conf => " + hiveSyncConfig.toString());
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
|
||||
}
|
||||
|
||||
public void syncHive(HiveConf conf) {
|
||||
this.conf = conf;
|
||||
syncHive();
|
||||
}
|
||||
|
||||
/**
|
||||
* Note that depending on configs and source-type, schemaProvider could either be eagerly or lazily created.
|
||||
* SchemaProvider creation is a precursor to HoodieWriteClient and AsyncCompactor creation. This method takes care of
|
||||
|
||||
@@ -19,7 +19,6 @@
|
||||
package org.apache.hudi.utilities.deltastreamer;
|
||||
|
||||
import com.beust.jcommander.Parameter;
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.client.utils.OperationConverter;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
@@ -28,6 +27,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.utilities.IdentitySplitter;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.utilities.UtilHelpers;
|
||||
@@ -131,7 +131,7 @@ public class HoodieMultiTableDeltaStreamer {
|
||||
Helpers.deepCopyConfigs(config, cfg);
|
||||
String overriddenTargetBasePath = tableProperties.getString(Constants.TARGET_BASE_PATH_PROP, "");
|
||||
cfg.targetBasePath = StringUtils.isNullOrEmpty(overriddenTargetBasePath) ? targetBasePath : overriddenTargetBasePath;
|
||||
if (cfg.enableMetaSync && StringUtils.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE().key(), ""))) {
|
||||
if (cfg.enableMetaSync && StringUtils.isNullOrEmpty(tableProperties.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), ""))) {
|
||||
throw new HoodieException("Meta sync table field not provided!");
|
||||
}
|
||||
populateSchemaProviderProps(cfg, tableProperties);
|
||||
|
||||
@@ -20,10 +20,13 @@ package org.apache.hudi.utilities;
|
||||
|
||||
import org.apache.hadoop.hive.metastore.api.MetaException;
|
||||
import org.apache.hadoop.hive.ql.metadata.HiveException;
|
||||
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.hive.HoodieHiveClient;
|
||||
import org.apache.hudi.hive.testutils.HiveTestUtil;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.utilities.exception.HoodieIncrementalPullSQLException;
|
||||
import org.junit.jupiter.api.AfterEach;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
@@ -38,7 +41,7 @@ import java.nio.file.Paths;
|
||||
import java.time.Instant;
|
||||
|
||||
import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem;
|
||||
import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncConfig;
|
||||
import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps;
|
||||
import static org.junit.jupiter.api.Assertions.assertDoesNotThrow;
|
||||
import static org.junit.jupiter.api.Assertions.assertThrows;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
@@ -69,12 +72,12 @@ public class TestHiveIncrementalPuller {
|
||||
}
|
||||
|
||||
private HiveIncrementalPuller.Config getHivePullerConfig(String incrementalSql) throws IOException {
|
||||
config.hiveJDBCUrl = hiveSyncConfig.jdbcUrl;
|
||||
config.hiveUsername = hiveSyncConfig.hiveUser;
|
||||
config.hivePassword = hiveSyncConfig.hivePass;
|
||||
config.hiveJDBCUrl = hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key());
|
||||
config.hiveUsername = hiveSyncProps.getString(HiveSyncConfig.HIVE_USER.key());
|
||||
config.hivePassword = hiveSyncProps.getString(HiveSyncConfig.HIVE_PASS.key());
|
||||
config.hoodieTmpDir = Files.createTempDirectory("hivePullerTest").toUri().toString();
|
||||
config.sourceDb = hiveSyncConfig.databaseName;
|
||||
config.sourceTable = hiveSyncConfig.tableName;
|
||||
config.sourceDb = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key());
|
||||
config.sourceTable = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key());
|
||||
config.targetDb = "tgtdb";
|
||||
config.targetTable = "test2";
|
||||
config.tmpDb = "tmp_db";
|
||||
@@ -98,9 +101,9 @@ public class TestHiveIncrementalPuller {
|
||||
private void createSourceTable() throws IOException, URISyntaxException {
|
||||
String instantTime = "101";
|
||||
HiveTestUtil.createCOWTable(instantTime, 5, true);
|
||||
hiveSyncConfig.syncMode = "jdbc";
|
||||
HiveTestUtil.hiveSyncConfig.batchSyncNum = 3;
|
||||
HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem);
|
||||
hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc");
|
||||
|
||||
HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem);
|
||||
tool.syncHoodieTable();
|
||||
}
|
||||
|
||||
@@ -113,20 +116,20 @@ public class TestHiveIncrementalPuller {
|
||||
tool.syncHoodieTable();
|
||||
}
|
||||
|
||||
private HiveSyncConfig getTargetHiveSyncConfig(String basePath) {
|
||||
HiveSyncConfig config = HiveSyncConfig.copy(hiveSyncConfig);
|
||||
config.databaseName = "tgtdb";
|
||||
config.tableName = "test2";
|
||||
config.basePath = basePath;
|
||||
config.batchSyncNum = 3;
|
||||
config.syncMode = "jdbc";
|
||||
return config;
|
||||
private TypedProperties getTargetHiveSyncConfig(String basePath) {
|
||||
TypedProperties targetHiveSyncProps = new TypedProperties(hiveSyncProps);
|
||||
targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), "tgtdb");
|
||||
targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), "test2");
|
||||
targetHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_BASE_PATH.key(), basePath);
|
||||
targetHiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc");
|
||||
|
||||
return targetHiveSyncProps;
|
||||
}
|
||||
|
||||
private HiveSyncConfig getAssertionSyncConfig(String databaseName) {
|
||||
HiveSyncConfig config = HiveSyncConfig.copy(hiveSyncConfig);
|
||||
config.databaseName = databaseName;
|
||||
return config;
|
||||
private TypedProperties getAssertionSyncConfig(String databaseName) {
|
||||
TypedProperties assertHiveSyncProps = new TypedProperties(hiveSyncProps);
|
||||
assertHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), databaseName);
|
||||
return assertHiveSyncProps;
|
||||
}
|
||||
|
||||
private void createTables() throws IOException, URISyntaxException {
|
||||
@@ -158,12 +161,11 @@ public class TestHiveIncrementalPuller {
|
||||
public void testPuller() throws IOException, URISyntaxException {
|
||||
createTables();
|
||||
HiveIncrementalPuller.Config cfg = getHivePullerConfig("select name from testdb.test1 where `_hoodie_commit_time` > '%s'");
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem);
|
||||
HoodieHiveClient hiveClient = new HoodieHiveClient(new HiveSyncConfig(hiveSyncProps), HiveTestUtil.getHiveConf(), fileSystem);
|
||||
hiveClient.createDatabase(cfg.tmpDb);
|
||||
HiveIncrementalPuller puller = new HiveIncrementalPuller(cfg);
|
||||
puller.saveDelta();
|
||||
HiveSyncConfig assertingConfig = getAssertionSyncConfig(cfg.tmpDb);
|
||||
HoodieHiveClient assertingClient = new HoodieHiveClient(assertingConfig, HiveTestUtil.getHiveConf(), fileSystem);
|
||||
HoodieHiveClient assertingClient = new HoodieHiveClient(new HiveSyncConfig(getAssertionSyncConfig(cfg.tmpDb)), HiveTestUtil.getHiveConf(), fileSystem);
|
||||
String tmpTable = cfg.targetTable + "__" + cfg.sourceTable;
|
||||
assertTrue(assertingClient.doesTableExist(tmpTable));
|
||||
}
|
||||
|
||||
@@ -18,7 +18,6 @@
|
||||
|
||||
package org.apache.hudi.utilities.functional;
|
||||
|
||||
import org.apache.hudi.DataSourceWriteOptions;
|
||||
import org.apache.hudi.common.config.TypedProperties;
|
||||
import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.WriteOperationType;
|
||||
@@ -28,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant;
|
||||
import org.apache.hudi.common.table.timeline.HoodieTimeline;
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.MultiPartKeysValueExtractor;
|
||||
import org.apache.hudi.utilities.schema.FilebasedSchemaProvider;
|
||||
import org.apache.hudi.utilities.testutils.UtilitiesTestBase;
|
||||
@@ -178,11 +178,11 @@ public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase {
|
||||
props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc");
|
||||
|
||||
// Hive Configs
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb1");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_TABLE().key(), "hive_trips");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb1");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_TABLE_NAME.key(), "hive_trips");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getName());
|
||||
UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE);
|
||||
}
|
||||
@@ -237,11 +237,11 @@ public class HoodieDeltaStreamerTestBase extends UtilitiesTestBase {
|
||||
|
||||
protected static void populateCommonHiveProps(TypedProperties props) {
|
||||
// Hive Configs
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_URL().key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_DATABASE().key(), "testdb2");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_ASSUME_DATE_PARTITION().key(), "false");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "datestr");
|
||||
props.setProperty(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(),
|
||||
props.setProperty(HiveSyncConfig.HIVE_URL.key(), "jdbc:hive2://127.0.0.1:9999/");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), "testdb2");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key(), "false");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "datestr");
|
||||
props.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(),
|
||||
MultiPartKeysValueExtractor.class.getName());
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user