diff --git a/docker/demo/sparksql-incremental.commands b/docker/demo/sparksql-incremental.commands index 5ea4729b932f2..da61347ec275b 100644 --- a/docker/demo/sparksql-incremental.commands +++ b/docker/demo/sparksql-incremental.commands @@ -21,8 +21,6 @@ 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; @@ -45,14 +43,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(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.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(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor"); @@ -77,14 +75,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(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.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(DataSourceWriteOptions.URL_ENCODE_PARTITIONING.key(), "true"). mode(SaveMode.Overwrite). save("/user/hive/warehouse/stock_ticks_derived_mor_bs"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 776c9066767c9..4847d9c91ed44 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -105,7 +105,7 @@ public class HoodieWriteConfig extends HoodieConfig { public static final String DELTASTREAMER_CHECKPOINT_KEY = "deltastreamer.checkpoint.key"; public static final ConfigProperty TBL_NAME = ConfigProperty - .key(HoodieTableConfig.HOODIE_TABLE_NAME_KEY) + .key("hoodie.table.name") .noDefaultValue() .withDocumentation("Table name that will be used for registering with metastores like HMS. Needs to be same across runs."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index 6b8103db3a663..bc8a5c443d39f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -82,8 +82,6 @@ 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 DATABASE_NAME = ConfigProperty .key("hoodie.database.name") @@ -92,7 +90,7 @@ public class HoodieTableConfig extends HoodieConfig { + "we can set it to limit the table name under a specific database"); public static final ConfigProperty NAME = ConfigProperty - .key(HOODIE_TABLE_NAME_KEY) + .key("hoodie.table.name") .noDefaultValue() .withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs."); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java index bc48661c88376..15e4aaa96f28a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ReflectionUtils.java @@ -92,24 +92,6 @@ public static Object loadClass(String clazz, Class[] constructorArgTypes, Obj } } - /** - * 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. */ diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java index 4736133f2c20c..bdde58adb19e6 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveQueryNode.java @@ -18,19 +18,17 @@ package org.apache.hudi.integ.testsuite.dag.nodes; -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; +import org.apache.hudi.DataSourceUtils; +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; /** * A hive query node in the DAG of operations for a workflow. used to perform a hive query with given config. @@ -48,14 +46,13 @@ public HiveQueryNode(DeltaConfig.Config config) { public void execute(ExecutionContext executionContext, int curItrCount) throws Exception { log.info("Executing hive query node {}", this.getName()); this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration()); - 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); + HiveSyncConfig hiveSyncConfig = DataSourceUtils + .buildHiveSyncConfig(executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getProps(), + executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getCfg().targetBasePath, + executionContext.getHoodieTestSuiteWriter().getDeltaStreamerWrapper() + .getDeltaSyncService().getDeltaSync().getCfg().baseFileFormat); this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter()); Connection con = DriverManager.getConnection(hiveSyncConfig.jdbcUrl, hiveSyncConfig.hiveUser, hiveSyncConfig.hivePass); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java index 3f0c90f03067c..97a1bee07c152 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/HiveSyncNode.java @@ -18,30 +18,31 @@ package org.apache.hudi.integ.testsuite.dag.nodes; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.hive.HiveSyncTool; +import org.apache.hudi.integ.testsuite.helpers.HiveServiceProvider; 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 { + 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"); - 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); + this.hiveServiceProvider.startLocalHiveServiceIfNeeded(executionContext.getHoodieTestSuiteWriter().getConfiguration()); + this.hiveServiceProvider.syncToLocalHiveIfNeeded(executionContext.getHoodieTestSuiteWriter()); + this.hiveServiceProvider.stopLocalHiveServiceIfNeeded(); + } + + public HiveServiceProvider getHiveServiceProvider() { + return hiveServiceProvider; } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java index a10e4d31fa736..85a292c2a2701 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/helpers/HiveServiceProvider.java @@ -21,9 +21,6 @@ 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; @@ -49,17 +46,12 @@ public void startLocalHiveServiceIfNeeded(Configuration configuration) throws IO } public void syncToLocalHiveIfNeeded(HoodieTestSuiteWriter writer) { - HiveSyncTool hiveSyncTool; if (this.config.isHiveLocal()) { - hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), - getLocalHiveServer().getHiveConf(), - FSUtils.getFs(writer.getWriteConfig().getBasePath(), getLocalHiveServer().getHiveConf())); + writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync() + .syncHive(getLocalHiveServer().getHiveConf()); } else { - hiveSyncTool = new HiveSyncTool(writer.getWriteConfig().getProps(), - getLocalHiveServer().getHiveConf(), - FSUtils.getFs(writer.getWriteConfig().getBasePath(), writer.getConfiguration())); + writer.getDeltaStreamerWrapper().getDeltaSyncService().getDeltaSync().syncHive(); } - hiveSyncTool.syncHoodieTable(); } public void stopLocalHiveServiceIfNeeded() throws IOException { diff --git a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java index 62c1d851e2108..1b0cef58c630e 100644 --- a/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java +++ b/hudi-integ-test/src/test/java/org/apache/hudi/integ/testsuite/job/TestHoodieTestSuiteJob.java @@ -22,7 +22,6 @@ 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; @@ -174,10 +173,10 @@ private static TypedProperties getProperties() { // Make path selection test suite specific props.setProperty("hoodie.deltastreamer.source.input.selector", DFSTestSuitePathSelector.class.getName()); // Hive Configs - 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.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(DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME().key(), TimestampBasedKeyGenerator.class.getName()); props.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider"); diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java index cc37de2f2926a..6a3843075bdaf 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java @@ -32,6 +32,8 @@ 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; @@ -57,6 +59,7 @@ 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; @@ -266,4 +269,32 @@ public static List getWriteStatuses(ControlMessage.ParticipantInfo 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; + } } diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java index 65a1d8ae2ddba..dae19cc413f3f 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/writers/KafkaConnectTransactionServices.java @@ -30,17 +30,22 @@ 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.HoodieSyncConfig; -import org.apache.hudi.sync.common.util.SyncUtilHelpers; +import org.apache.hudi.sync.common.AbstractSyncTool; 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; @@ -49,6 +54,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.Set; /** @@ -161,10 +167,43 @@ private void syncMeta() { if (connectConfigs.isMetaSyncEnabled()) { Set syncClientToolClasses = new HashSet<>( Arrays.asList(connectConfigs.getMetaSyncClasses().split(","))); - FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration()); for (String impl : syncClientToolClasses) { - SyncUtilHelpers.runHoodieMetaSync(impl.trim(), connectConfigs.getProps(), hadoopConf, fs, tableBasePath, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue()); + 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(); + } } } } + + 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(); + } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java index 4042f431d7d56..634389b1e68b1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -42,7 +42,6 @@ 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; @@ -271,11 +270,6 @@ public static JavaRDD dropDuplicates(JavaSparkContext jssc, JavaRD 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(); @@ -316,7 +310,7 @@ public static HiveSyncConfig buildHiveSyncConfig(TypedProperties props, String b 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(), - DataSourceWriteOptions.HIVE_SYNC_BUCKET_SYNC().defaultValue()) + (boolean) 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())) { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala index f86e55b436f1e..530e435783696 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -25,10 +25,9 @@ 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.{HiveSyncConfig, HiveSyncTool, MultiPartKeysValueExtractor, NonPartitionedExtractor, SlashEncodedDayPartitionValueExtractor} +import org.apache.hudi.hive.{HiveStylePartitionValueExtractor, 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} @@ -247,7 +246,7 @@ object DataSourceWriteOptions { } val TABLE_NAME: ConfigProperty[String] = ConfigProperty - .key(HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY) + .key("hoodie.datasource.write.table.name") .noDefaultValue() .withDocumentation("Table name for the datasource write. Also used to register the table into meta stores.") @@ -381,79 +380,185 @@ 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 - /** - * @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 + 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 Use {@link HIVE_SYNC_MODE} instead of this config from 0.9.0 */ @Deprecated - 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 + 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.") /** * 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. */ - @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 + 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("") // Create table as managed 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; + 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.") // Async Compaction - Enabled by default for MOR val ASYNC_COMPACT_ENABLE: ConfigProperty[String] = ConfigProperty @@ -479,19 +584,19 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */ @Deprecated - val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.key() + val HIVE_ASSUME_DATE_PARTITION_OPT_KEY = HIVE_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 = HiveSyncConfig.HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() + val HIVE_USE_PRE_APACHE_INPUT_FORMAT_OPT_KEY = HIVE_USE_PRE_APACHE_INPUT_FORMAT.key() /** @deprecated Use {@link HIVE_USE_JDBC} and its methods instead */ @Deprecated - val HIVE_USE_JDBC_OPT_KEY = HiveSyncConfig.HIVE_USE_JDBC.key() + val HIVE_USE_JDBC_OPT_KEY = HIVE_USE_JDBC.key() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key() + val HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.key() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key() + val HIVE_IGNORE_EXCEPTIONS_OPT_KEY = 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() @@ -506,34 +611,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 = HiveSyncConfig.HIVE_SYNC_ENABLED.key() + val HIVE_SYNC_ENABLED_OPT_KEY = HIVE_SYNC_ENABLED.key() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated - val META_SYNC_ENABLED_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() + val META_SYNC_ENABLED_OPT_KEY = META_SYNC_ENABLED.key() /** @deprecated Use {@link HIVE_DATABASE} and its methods instead */ @Deprecated - val HIVE_DATABASE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() + val HIVE_DATABASE_OPT_KEY = HIVE_DATABASE.key() /** @deprecated Use {@link HIVE_TABLE} and its methods instead */ @Deprecated - val HIVE_TABLE_OPT_KEY = HoodieSyncConfig.META_SYNC_DATABASE_NAME.key() + val HIVE_TABLE_OPT_KEY = HIVE_TABLE.key() /** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */ @Deprecated - val HIVE_BASE_FILE_FORMAT_OPT_KEY = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.key() + val HIVE_BASE_FILE_FORMAT_OPT_KEY = HIVE_BASE_FILE_FORMAT.key() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val HIVE_USER_OPT_KEY = HiveSyncConfig.HIVE_USER.key() + val HIVE_USER_OPT_KEY = HIVE_USER.key() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val HIVE_PASS_OPT_KEY = HiveSyncConfig.HIVE_PASS.key() + val HIVE_PASS_OPT_KEY = HIVE_PASS.key() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val HIVE_URL_OPT_KEY = HiveSyncConfig.HIVE_URL.key() + val HIVE_URL_OPT_KEY = HIVE_URL.key() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated - val HIVE_PARTITION_FIELDS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key() + val HIVE_PARTITION_FIELDS_OPT_KEY = HIVE_PARTITION_FIELDS.key() /** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */ @Deprecated - val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key() + val HIVE_PARTITION_EXTRACTOR_CLASS_OPT_KEY = HIVE_PARTITION_EXTRACTOR_CLASS.key() /** @deprecated Use {@link KEYGENERATOR_CLASS_NAME} and its methods instead */ @Deprecated @@ -643,60 +748,60 @@ object DataSourceWriteOptions { /** @deprecated Use {@link HIVE_SYNC_ENABLED} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HiveSyncConfig.HIVE_SYNC_ENABLED.defaultValue() + val DEFAULT_HIVE_SYNC_ENABLED_OPT_VAL = HIVE_SYNC_ENABLED.defaultValue() /** @deprecated Use {@link META_SYNC_ENABLED} and its methods instead */ @Deprecated - val DEFAULT_META_SYNC_ENABLED_OPT_VAL = HoodieSyncConfig.META_SYNC_ENABLED.defaultValue() + val DEFAULT_META_SYNC_ENABLED_OPT_VAL = META_SYNC_ENABLED.defaultValue() /** @deprecated Use {@link HIVE_DATABASE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_DATABASE_OPT_VAL = HoodieSyncConfig.META_SYNC_DATABASE_NAME.defaultValue() + val DEFAULT_HIVE_DATABASE_OPT_VAL = HIVE_DATABASE.defaultValue() /** @deprecated Use {@link HIVE_TABLE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_TABLE_OPT_VAL = HoodieSyncConfig.META_SYNC_TABLE_NAME.defaultValue() + val DEFAULT_HIVE_TABLE_OPT_VAL = HIVE_TABLE.defaultValue() /** @deprecated Use {@link HIVE_BASE_FILE_FORMAT} and its methods instead */ @Deprecated - val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue() + val DEFAULT_HIVE_BASE_FILE_FORMAT_OPT_VAL = HIVE_BASE_FILE_FORMAT.defaultValue() /** @deprecated Use {@link HIVE_USER} and its methods instead */ @Deprecated - val DEFAULT_HIVE_USER_OPT_VAL = HiveSyncConfig.HIVE_USER.defaultValue() + val DEFAULT_HIVE_USER_OPT_VAL = HIVE_USER.defaultValue() /** @deprecated Use {@link HIVE_PASS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PASS_OPT_VAL = HiveSyncConfig.HIVE_PASS.defaultValue() + val DEFAULT_HIVE_PASS_OPT_VAL = HIVE_PASS.defaultValue() /** @deprecated Use {@link HIVE_URL} and its methods instead */ @Deprecated - val DEFAULT_HIVE_URL_OPT_VAL = HiveSyncConfig.HIVE_URL.defaultValue() + val DEFAULT_HIVE_URL_OPT_VAL = HIVE_URL.defaultValue() /** @deprecated Use {@link HIVE_PARTITION_FIELDS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.defaultValue() + val DEFAULT_HIVE_PARTITION_FIELDS_OPT_VAL = HIVE_PARTITION_FIELDS.defaultValue() /** @deprecated Use {@link HIVE_PARTITION_EXTRACTOR_CLASS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.defaultValue() + val DEFAULT_HIVE_PARTITION_EXTRACTOR_CLASS_OPT_VAL = HIVE_PARTITION_EXTRACTOR_CLASS.defaultValue() /** @deprecated Use {@link HIVE_ASSUME_DATE_PARTITION} and its methods instead */ @Deprecated - val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION.defaultValue() + val DEFAULT_HIVE_ASSUME_DATE_PARTITION_OPT_VAL = HIVE_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 = HiveSyncConfig.HIVE_USE_JDBC.defaultValue() + val DEFAULT_HIVE_USE_JDBC_OPT_VAL = HIVE_USE_JDBC.defaultValue() /** @deprecated Use {@link HIVE_AUTO_CREATE_DATABASE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.defaultValue() + val DEFAULT_HIVE_AUTO_CREATE_DATABASE_OPT_KEY = HIVE_AUTO_CREATE_DATABASE.defaultValue() /** @deprecated Use {@link HIVE_IGNORE_EXCEPTIONS} and its methods instead */ @Deprecated - val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.defaultValue() + val DEFAULT_HIVE_IGNORE_EXCEPTIONS_OPT_KEY = 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 = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() + val HIVE_SKIP_RO_SUFFIX = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key() /** @deprecated Use {@link HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HiveSyncConfig.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() + val DEFAULT_HIVE_SKIP_RO_SUFFIX_VAL = HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.defaultValue() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key() + val HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.key() /** @deprecated Use {@link HIVE_SUPPORT_TIMESTAMP_TYPE} and its methods instead */ @Deprecated - val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() + val DEFAULT_HIVE_SUPPORT_TIMESTAMP = HIVE_SUPPORT_TIMESTAMP_TYPE.defaultValue() /** @deprecated Use {@link ASYNC_COMPACT_ENABLE} and its methods instead */ @Deprecated val ASYNC_COMPACT_ENABLE_OPT_KEY = ASYNC_COMPACT_ENABLE.key() diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index fc83cebc945d4..8ba0db907b370 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -19,6 +19,7 @@ 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 @@ -31,7 +32,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, StringUtils} +import org.apache.hudi.common.util.{CommitUtils, ReflectionUtils, StringUtils} import org.apache.hudi.config.HoodieBootstrapConfig.{BASE_PATH, INDEX_CLASS_NAME} import org.apache.hudi.config.{HoodieInternalConfig, HoodieWriteConfig} import org.apache.hudi.exception.HoodieException @@ -39,21 +40,24 @@ 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.factory.HoodieSparkKeyGeneratorFactory 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.keygen.factory.HoodieSparkKeyGeneratorFactory +import org.apache.hudi.sync.common.AbstractSyncTool 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._ -import org.apache.spark.sql.internal.StaticSQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.types.StructType -import org.apache.spark.{SPARK_VERSION, SparkContext} +import org.apache.spark.sql._ +import org.apache.spark.SparkContext +import java.util.Properties import scala.collection.JavaConversions._ import scala.collection.mutable +import scala.collection.mutable.ListBuffer object HoodieSparkSqlWriter { @@ -508,7 +512,14 @@ object HoodieSparkSqlWriter { + " To use row writer please switch to spark 2 or spark 3") } val hoodieConfig = HoodieWriterUtils.convertMapToHoodieConfig(params) - val syncHiveSuccess = metaSync(sqlContext.sparkSession, hoodieConfig, basePath, df.schema) + 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 + } (syncHiveSuccess, common.util.Option.ofNullable(instantTime)) } @@ -547,10 +558,57 @@ 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(HiveSyncConfig.HIVE_SYNC_ENABLED).toBoolean - var metaSyncEnabled = hoodieConfig.getStringOrDefault(HoodieSyncConfig.META_SYNC_ENABLED).toBoolean + val hiveSyncEnabled = hoodieConfig.getStringOrDefault(HIVE_SYNC_ENABLED).toBoolean + var metaSyncEnabled = hoodieConfig.getStringOrDefault(META_SYNC_ENABLED).toBoolean var syncClientToolClassSet = scala.collection.mutable.Set[String]() hoodieConfig.getString(META_SYNC_CLIENT_TOOL_CLASS_NAME).split(",").foreach(syncClass => syncClientToolClassSet += syncClass) @@ -559,23 +617,29 @@ 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 => { - SyncUtilHelpers.runHoodieMetaSync(impl.trim, properties, hiveConf, fs, basePath.toString, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue) + 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 }) } - true + metaSyncSuccess } /** diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala index a4d76763fc558..8a4ad9d85d72d 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieWriterUtils.scala @@ -18,14 +18,13 @@ 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 @@ -65,21 +64,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(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_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(HIVE_STYLE_PARTITIONING) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_USE_JDBC) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE) - hoodieConfig.setDefaultValue(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE) + hoodieConfig.setDefaultValue(HIVE_USE_JDBC) + hoodieConfig.setDefaultValue(HIVE_CREATE_MANAGED_TABLE) + hoodieConfig.setDefaultValue(HIVE_SYNC_AS_DATA_SOURCE_TABLE) hoodieConfig.setDefaultValue(ASYNC_COMPACT_ENABLE) hoodieConfig.setDefaultValue(INLINE_CLUSTERING_ENABLE) hoodieConfig.setDefaultValue(ASYNC_CLUSTERING_ENABLE) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala index 2e639d78e1e17..7f3cedbd3986e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableDropPartitionCommand.scala @@ -22,9 +22,8 @@ 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.{HiveSyncConfig, MultiPartKeysValueExtractor} +import org.apache.hudi.hive.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 @@ -103,15 +102,15 @@ case class AlterHoodieTableDropPartitionCommand( RECORDKEY_FIELD.key -> hoodieCatalogTable.primaryKeys.mkString(","), PRECOMBINE_FIELD.key -> hoodieCatalogTable.preCombineKey.getOrElse(""), PARTITIONPATH_FIELD.key -> partitionFields, - 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 + 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 ) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala index 1d2cea10afa7d..2877dd8d9ee94 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/CreateHoodieTableAsSelectCommand.scala @@ -20,7 +20,6 @@ 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} @@ -81,9 +80,9 @@ case class CreateHoodieTableAsSelectCommand( val tblProperties = hoodieCatalogTable.catalogProperties val options = Map( - 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.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), DataSourceWriteOptions.SQL_INSERT_MODE.key -> InsertMode.NON_STRICT.value(), DataSourceWriteOptions.SQL_ENABLE_BULK_INSERT.key -> "true" ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala index f6da1b3868302..fff8f9194562f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/DeleteHoodieTableCommand.scala @@ -20,7 +20,6 @@ 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._ @@ -76,8 +75,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, - HiveSyncConfig.HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), - HiveSyncConfig.HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", + HIVE_SYNC_MODE.key -> HiveSyncMode.HMS.name(), + HIVE_SUPPORT_TIMESTAMP_TYPE.key -> "true", HoodieWriteConfig.DELETE_PARALLELISM_VALUE.key -> "200", SqlKeyGenerator.PARTITION_SCHEMA -> partitionSchema.toDDL ) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala index 74d6226b49f6d..1d9aedd2af6fa 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/MergeIntoHoodieTableCommand.scala @@ -22,9 +22,8 @@ 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.{HiveSyncConfig, MultiPartKeysValueExtractor} +import org.apache.hudi.hive.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 @@ -463,14 +462,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, - 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, + 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, 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", diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java index d25896bdd9779..b6e595c40a8df 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaApp.java @@ -24,7 +24,6 @@ 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; @@ -256,24 +255,24 @@ public void run() throws Exception { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - 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"); + 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"); if (nonPartitionedTable) { writer = writer - .option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), NonPartitionedExtractor.class.getCanonicalName()) .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), ""); } else if (useMultiPartitionKeys) { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java index 42c6a4fd89f41..8302ece4b9ae9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaGenerateApp.java @@ -23,7 +23,6 @@ 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; @@ -126,24 +125,24 @@ private HoodieTestDataGenerator getDataGenerate() { private DataFrameWriter updateHiveSyncConfig(DataFrameWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - 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"); + 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"); if (nonPartitionedTable) { writer = writer - .option(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + .option(DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), NonPartitionedExtractor.class.getCanonicalName()) .option(DataSourceWriteOptions.PARTITIONPATH_FIELD().key(), ""); } else if (useMultiPartitionKeys) { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java index 207a9492fdad0..4af3943966aa3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/HoodieJavaStreamingApp.java @@ -27,7 +27,6 @@ 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; @@ -383,19 +382,19 @@ public void stream(Dataset streamingInput, String operationType, String che private DataStreamWriter updateHiveSyncConfig(DataStreamWriter writer) { if (enableHiveSync) { LOG.info("Enabling Hive sync to " + hiveJdbcUrl); - 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"); + 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"); if (useMultiPartitionKeys) { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "year,month,day").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), MultiPartKeysValueExtractor.class.getCanonicalName()); } else { - writer = writer.option(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "dateStr").option( - HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), + writer = writer.option(DataSourceWriteOptions.HIVE_PARTITION_FIELDS().key(), "dateStr").option( + DataSourceWriteOptions.HIVE_PARTITION_EXTRACTOR_CLASS().key(), SlashEncodedDayPartitionValueExtractor.class.getCanonicalName()); } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala index 9920aa80baf09..d5c3bfa01fc2e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestDataSourceOptions.scala @@ -20,7 +20,6 @@ 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 @@ -32,10 +31,10 @@ class TestDataSourceOptions { ) val modifiedOptions1 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions1) assertEquals(classOf[ComplexKeyGenerator].getName, modifiedOptions1(KEYGENERATOR_CLASS_NAME.key)) - assertEquals("hudi_table", modifiedOptions1(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) - assertEquals("year,month", modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key)) + assertEquals("hudi_table", modifiedOptions1(HIVE_TABLE.key)) + assertEquals("year,month", modifiedOptions1(HIVE_PARTITION_FIELDS.key)) assertEquals(classOf[MultiPartKeysValueExtractor].getName, - modifiedOptions1(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key)) + modifiedOptions1(HIVE_PARTITION_EXTRACTOR_CLASS.key)) val inputOptions2 = Map( TABLE_NAME.key -> "hudi_table", @@ -44,9 +43,9 @@ class TestDataSourceOptions { ) val modifiedOptions2 = HoodieWriterUtils.parametersWithWriteDefaults(inputOptions2) assertEquals(classOf[SimpleKeyGenerator].getName, modifiedOptions2(KEYGENERATOR_CLASS_NAME.key)) - assertEquals("hudi_table", modifiedOptions2(HoodieSyncConfig.META_SYNC_TABLE_NAME.key)) - assertEquals("year", modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_FIELDS.key)) + assertEquals("hudi_table", modifiedOptions2(HIVE_TABLE.key)) + assertEquals("year", modifiedOptions2(HIVE_PARTITION_FIELDS.key)) assertEquals(classOf[HiveStylePartitionValueExtractor].getName, - modifiedOptions2(HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key)) + modifiedOptions2(HIVE_PARTITION_EXTRACTOR_CLASS.key)) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala index c14d0bb063d7b..b5186fb1ac089 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala @@ -664,6 +664,55 @@ 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. */ diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java index 2088d48d8a383..bf0369ae2ee58 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/DLASyncTool.java @@ -23,8 +23,6 @@ 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; @@ -43,6 +41,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Properties; import java.util.stream.Collectors; /** @@ -64,8 +63,8 @@ public class DLASyncTool extends AbstractSyncTool { private final String snapshotTableName; private final Option roTableTableName; - public DLASyncTool(TypedProperties properties, Configuration conf, FileSystem fs) { - super(properties, conf, fs); + public DLASyncTool(Properties properties, FileSystem fs) { + super(properties, fs); this.hoodieDLAClient = new HoodieDLAClient(Utils.propertiesToConfig(properties), fs); this.cfg = Utils.propertiesToConfig(properties); switch (hoodieDLAClient.getTableType()) { @@ -206,8 +205,7 @@ public static void main(String[] args) { cmd.usage(); System.exit(1); } - Configuration hadoopConf = new Configuration(); - FileSystem fs = FSUtils.getFs(cfg.basePath, hadoopConf); - new DLASyncTool(Utils.configToProperties(cfg), hadoopConf, fs).syncHoodieTable(); + FileSystem fs = FSUtils.getFs(cfg.basePath, new Configuration()); + new DLASyncTool(Utils.configToProperties(cfg), fs).syncHoodieTable(); } } diff --git a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java index d1b0dd4e9d56f..ad47b71f84948 100644 --- a/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java +++ b/hudi-sync/hudi-dla-sync/src/main/java/org/apache/hudi/dla/util/Utils.java @@ -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 TypedProperties configToProperties(DLASyncConfig cfg) { - TypedProperties properties = new TypedProperties(); + public static Properties configToProperties(DLASyncConfig cfg) { + Properties properties = new Properties(); 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 static TypedProperties configToProperties(DLASyncConfig cfg) { return properties; } - public static DLASyncConfig propertiesToConfig(TypedProperties properties) { + public static DLASyncConfig propertiesToConfig(Properties properties) { DLASyncConfig config = new DLASyncConfig(); config.databaseName = properties.getProperty(DLA_DATABASE_OPT_KEY); config.tableName = properties.getProperty(DLA_TABLE_OPT_KEY); diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java index eb4fc62d40d6d..8ee9daa836648 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncConfig.java @@ -18,16 +18,27 @@ package org.apache.hudi.hive; -import org.apache.hudi.common.config.ConfigProperty; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.sync.common.HoodieSyncConfig; +import org.apache.hudi.common.config.HoodieMetadataConfig; import com.beust.jcommander.Parameter; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + /** - * Configs needed to sync data into the Hive Metastore. + * Configs needed to sync data into Hive. */ -public class HiveSyncConfig extends HoodieSyncConfig { +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"; @Parameter(names = {"--user"}, description = "Hive username") public String hiveUser; @@ -41,31 +52,48 @@ public class HiveSyncConfig extends HoodieSyncConfig { @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 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; + public Boolean usePreApacheInputFormat = false; @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; + public Boolean useJdbc = true; @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; + public Boolean autoCreateDatabase = true; @Parameter(names = {"--ignore-exceptions"}, description = "Ignore hive exceptions") - public Boolean ignoreExceptions; + public Boolean ignoreExceptions = false; @Parameter(names = {"--skip-ro-suffix"}, description = "Skip the `_ro` suffix for Read optimized table, when registering") - public Boolean skipROSuffix; + 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; @Parameter(names = {"--table-properties"}, description = "Table properties to hive table") public String tableProperties; @@ -78,170 +106,64 @@ public class HiveSyncConfig extends HoodieSyncConfig { @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; + public Boolean supportTimestamp = false; + + @Parameter(names = {"--decode-partition"}, description = "Decode the partition value if the partition has encoded during writing") + public Boolean decodePartition = false; @Parameter(names = {"--managed-table"}, description = "Create a managed table") - public Boolean createManagedTable; + public Boolean createManagedTable = false; @Parameter(names = {"--batch-sync-num"}, description = "The number of partitions one batch when synchronous partitions to hive") - public Integer batchSyncNum; + public Integer batchSyncNum = 1000; @Parameter(names = {"--spark-datasource"}, description = "Whether sync this table as spark data source table.") - public Boolean syncAsSparkDataSourceTable; + public Boolean syncAsSparkDataSourceTable = true; @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; + public int sparkSchemaLengthThreshold = 4000; @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; - // 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 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 HIVE_USER = ConfigProperty - .key("hoodie.datasource.hive_sync.username") - .defaultValue("hive") - .withDocumentation("hive user name to use"); - - public static final ConfigProperty HIVE_PASS = ConfigProperty - .key("hoodie.datasource.hive_sync.password") - .defaultValue("hive") - .withDocumentation("hive password to use"); - - public static final ConfigProperty HIVE_URL = ConfigProperty - .key("hoodie.datasource.hive_sync.jdbcurl") - .defaultValue("jdbc:hive2://localhost:10000") - .withDocumentation("Hive metastore url"); - - public static final ConfigProperty 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 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 METASTORE_URIS = ConfigProperty - .key("hoodie.datasource.hive_sync.metastore.uris") - .defaultValue("thrift://localhost:9083") - .withDocumentation("Hive metastore url"); - - public static final ConfigProperty 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 HIVE_IGNORE_EXCEPTIONS = ConfigProperty - .key("hoodie.datasource.hive_sync.ignore_exceptions") - .defaultValue("false") - .withDocumentation("Ignore exceptions when syncing with Hive."); - - public static final ConfigProperty 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 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 HIVE_TABLE_PROPERTIES = ConfigProperty - .key("hoodie.datasource.hive_sync.table_properties") - .noDefaultValue() - .withDocumentation("Additional properties to store with table."); - - public static final ConfigProperty HIVE_TABLE_SERDE_PROPERTIES = ConfigProperty - .key("hoodie.datasource.hive_sync.serde_properties") - .noDefaultValue() - .withDocumentation("Serde properties to hive table."); - - public static final ConfigProperty HIVE_SYNC_AS_DATA_SOURCE_TABLE = ConfigProperty - .key("hoodie.datasource.hive_sync.sync_as_datasource") - .defaultValue("true") - .withDocumentation(""); - - public static final ConfigProperty 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 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 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 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 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 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 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); + // 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; } @Override @@ -275,7 +197,6 @@ public String toString() { + ", sparkSchemaLengthThreshold=" + sparkSchemaLengthThreshold + ", withOperationField=" + withOperationField + ", isConditionalSync=" + isConditionalSync - + ", sparkVersion=" + sparkVersion + ", syncComment=" + syncComment + '}'; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java index cac70ab5446e4..952742b913330 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/HiveSyncTool.java @@ -25,8 +25,6 @@ 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; @@ -72,53 +70,40 @@ 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 hiveSyncConfig; + protected final HiveSyncConfig cfg; protected HoodieHiveClient hoodieHiveClient = null; protected String snapshotTableName = null; protected Option roTableName = null; - 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); - } + public HiveSyncTool(HiveSyncConfig cfg, HiveConf configuration, FileSystem fs) { + super(configuration.getAllProperties(), fs); - private void init(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { try { - if (StringUtils.isNullOrEmpty(hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname))) { - hiveConf.set(HiveConf.ConfVars.METASTOREURIS.varname, hiveSyncConfig.metastoreUris); - } - this.hoodieHiveClient = new HoodieHiveClient(hiveSyncConfig, hiveConf, fs); + this.hoodieHiveClient = new HoodieHiveClient(cfg, configuration, fs); } catch (RuntimeException e) { - if (hiveSyncConfig.ignoreExceptions) { + if (cfg.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(hiveSyncConfig.partitionValueExtractorClass)) { + if (NonPartitionedExtractor.class.getName().equals(cfg.partitionValueExtractorClass)) { LOG.warn("Set partitionFields to empty, since the NonPartitionedExtractor is used"); - hiveSyncConfig.partitionFields = new ArrayList<>(); + cfg.partitionFields = new ArrayList<>(); } if (hoodieHiveClient != null) { switch (hoodieHiveClient.getTableType()) { case COPY_ON_WRITE: - this.snapshotTableName = hiveSyncConfig.tableName; + this.snapshotTableName = cfg.tableName; this.roTableName = Option.empty(); break; case MERGE_ON_READ: - this.snapshotTableName = hiveSyncConfig.tableName + SUFFIX_SNAPSHOT_TABLE; - this.roTableName = hiveSyncConfig.skipROSuffix ? Option.of(hiveSyncConfig.tableName) : - Option.of(hiveSyncConfig.tableName + SUFFIX_READ_OPTIMIZED_TABLE); + this.snapshotTableName = cfg.tableName + SUFFIX_SNAPSHOT_TABLE; + this.roTableName = cfg.skipROSuffix ? Option.of(cfg.tableName) : + Option.of(cfg.tableName + SUFFIX_READ_OPTIMIZED_TABLE); break; default: LOG.error("Unknown table type " + hoodieHiveClient.getTableType()); @@ -131,13 +116,10 @@ private void init(HiveSyncConfig hiveSyncConfig, HiveConf hiveConf) { 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 " + hiveSyncConfig.tableName, re); + throw new HoodieException("Got runtime exception when hive syncing " + cfg.tableName, re); } finally { if (hoodieHiveClient != null) { hoodieHiveClient.close(); @@ -168,19 +150,18 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, + " of type " + hoodieHiveClient.getTableType()); // check if the database exists else create it - if (hiveSyncConfig.autoCreateDatabase) { + if (cfg.autoCreateDatabase) { try { - if (!hoodieHiveClient.doesDataBaseExist(hiveSyncConfig.databaseName)) { - hoodieHiveClient.createDatabase(hiveSyncConfig.databaseName); + if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) { + hoodieHiveClient.createDatabase(cfg.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(hiveSyncConfig.databaseName)) { - LOG.error("Hive database does not exist " + hiveSyncConfig.databaseName); - throw new HoodieHiveSyncException("hive database does not exist " + hiveSyncConfig.databaseName); + if (!hoodieHiveClient.doesDataBaseExist(cfg.databaseName)) { + throw new HoodieHiveSyncException("hive database does not exist " + cfg.databaseName); } } @@ -200,7 +181,7 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, if (hoodieHiveClient.isBootstrap() && hoodieHiveClient.getTableType() == HoodieTableType.MERGE_ON_READ && !readAsOptimized) { - hiveSyncConfig.syncAsSparkDataSourceTable = false; + cfg.syncAsSparkDataSourceTable = false; } // Sync schema if needed @@ -219,7 +200,7 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, // Sync the partitions if needed boolean partitionsChanged = syncPartitions(tableName, writtenPartitionsSince, isDropPartition); boolean meetSyncConditions = schemaChanged || partitionsChanged; - if (!hiveSyncConfig.isConditionalSync || meetSyncConditions) { + if (!cfg.isConditionalSync || meetSyncConditions) { hoodieHiveClient.updateLastCommitTimeSynced(tableName); } LOG.info("Sync complete for " + tableName); @@ -235,10 +216,10 @@ protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, private boolean syncSchema(String tableName, boolean tableExists, boolean useRealTimeInputFormat, boolean readAsOptimized, MessageType schema) { // Append spark table properties & serde properties - Map tableProperties = ConfigUtils.toMap(hiveSyncConfig.tableProperties); - Map serdeProperties = ConfigUtils.toMap(hiveSyncConfig.serdeProperties); - if (hiveSyncConfig.syncAsSparkDataSourceTable) { - Map sparkTableProperties = getSparkTableProperties(hiveSyncConfig.sparkSchemaLengthThreshold, schema); + Map tableProperties = ConfigUtils.toMap(cfg.tableProperties); + Map serdeProperties = ConfigUtils.toMap(cfg.serdeProperties); + if (cfg.syncAsSparkDataSourceTable) { + Map sparkTableProperties = getSparkTableProperties(cfg.sparkSchemaLengthThreshold, schema); Map sparkSerdeProperties = getSparkSerdeProperties(readAsOptimized); tableProperties.putAll(sparkTableProperties); serdeProperties.putAll(sparkSerdeProperties); @@ -247,10 +228,10 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea // Check and sync schema if (!tableExists) { LOG.info("Hive table " + tableName + " is not found. Creating it"); - HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(hiveSyncConfig.baseFileFormat.toUpperCase()); + HoodieFileFormat baseFileFormat = HoodieFileFormat.valueOf(cfg.baseFileFormat.toUpperCase()); String inputFormatClassName = HoodieInputFormatUtils.getInputFormatClassName(baseFileFormat, useRealTimeInputFormat); - if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && hiveSyncConfig.usePreApacheInputFormat) { + if (baseFileFormat.equals(HoodieFileFormat.PARQUET) && cfg.usePreApacheInputFormat) { // Parquet input format had an InputFormat class visible under the old naming scheme. inputFormatClassName = useRealTimeInputFormat ? com.uber.hoodie.hadoop.realtime.HoodieRealtimeInputFormat.class.getName() @@ -269,12 +250,12 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea } else { // Check if the table schema has evolved Map tableSchema = hoodieHiveClient.getTableSchema(tableName); - SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, hiveSyncConfig.partitionFields, hiveSyncConfig.supportTimestamp); + SchemaDifference schemaDiff = HiveSchemaUtil.getSchemaDifference(schema, tableSchema, cfg.partitionFields, cfg.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 (hiveSyncConfig.tableProperties != null || hiveSyncConfig.syncAsSparkDataSourceTable) { + if (cfg.tableProperties != null || cfg.syncAsSparkDataSourceTable) { hoodieHiveClient.updateTableProperties(tableName, tableProperties); LOG.info("Sync table properties for " + tableName + ", table properties is: " + tableProperties); } @@ -284,7 +265,7 @@ private boolean syncSchema(String tableName, boolean tableExists, boolean useRea } } - if (hiveSyncConfig.syncComment) { + if (cfg.syncComment) { Schema avroSchemaWithoutMetadataFields = hoodieHiveClient.getAvroSchemaWithoutMetadataFields(); Map newComments = avroSchemaWithoutMetadataFields.getFields() .stream().collect(Collectors.toMap(Schema.Field::name, field -> StringUtils.isNullOrEmpty(field.doc()) ? "" : field.doc())); @@ -309,7 +290,7 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M // 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 partitionNames = hiveSyncConfig.partitionFields; + List partitionNames = cfg.partitionFields; List partitionCols = new ArrayList<>(); List dataCols = new ArrayList<>(); Map column2Field = new HashMap<>(); @@ -338,8 +319,8 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M Map sparkProperties = new HashMap<>(); sparkProperties.put("spark.sql.sources.provider", "hudi"); - if (!StringUtils.isNullOrEmpty(hiveSyncConfig.sparkVersion)) { - sparkProperties.put("spark.sql.create.version", hiveSyncConfig.sparkVersion); + if (!StringUtils.isNullOrEmpty(cfg.sparkVersion)) { + sparkProperties.put("spark.sql.create.version", cfg.sparkVersion); } // Split the schema string to multi-parts according the schemaLengthThreshold size. String schemaString = Parquet2SparkSchemaUtils.convertToSparkSchemaJson(reOrderedType); @@ -363,7 +344,7 @@ private Map getSparkTableProperties(int schemaLengthThreshold, M private Map getSparkSerdeProperties(boolean readAsOptimized) { Map sparkSerdeProperties = new HashMap<>(); - sparkSerdeProperties.put("path", hiveSyncConfig.basePath); + sparkSerdeProperties.put("path", cfg.basePath); sparkSerdeProperties.put(ConfigUtils.IS_QUERY_AS_RO_TABLE, String.valueOf(readAsOptimized)); return sparkSerdeProperties; } diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java index 16c30a16aabc6..4bafd854ae318 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncConfig.java @@ -18,24 +18,15 @@ package org.apache.hudi.hive.replication; -import org.apache.hudi.common.config.TypedProperties; -import org.apache.hudi.hive.HiveSyncConfig; - import com.beust.jcommander.Parameter; +import org.apache.hudi.hive.HiveSyncConfig; 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(cfg.getProps()); + GlobalHiveSyncConfig newConfig = new GlobalHiveSyncConfig(); newConfig.basePath = cfg.basePath; newConfig.assumeDatePartitioning = cfg.assumeDatePartitioning; newConfig.databaseName = cfg.databaseName; diff --git a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java index 51b2a77ae7433..1d225cb840c05 100644 --- a/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/main/java/org/apache/hudi/hive/replication/GlobalHiveSyncTool.java @@ -48,9 +48,9 @@ public void syncHoodieTable() { @Override protected void syncHoodieTable(String tableName, boolean useRealtimeInputFormat, boolean readAsOptimized) { super.syncHoodieTable(tableName, useRealtimeInputFormat, readAsOptimized); - if (((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp != null) { + if (((GlobalHiveSyncConfig)cfg).globallyReplicatedTimeStamp != null) { hoodieHiveClient.updateLastReplicatedTimeStamp(tableName, - ((GlobalHiveSyncConfig) hiveSyncConfig).globallyReplicatedTimeStamp); + ((GlobalHiveSyncConfig) cfg).globallyReplicatedTimeStamp); } LOG.info("Sync complete for " + tableName); } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java index 0e23615d5dadd..9fc87fcb456b0 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/TestHiveSyncTool.java @@ -59,15 +59,12 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.hudi.hive.testutils.HiveTestUtil.basePath; import static org.apache.hudi.hive.testutils.HiveTestUtil.ddlExecutor; import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; -import static org.apache.hudi.hive.testutils.HiveTestUtil.getHiveConf; -import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; +import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncConfig; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -82,7 +79,6 @@ private static Iterable syncMode() { return SYNC_MODES; } - // useSchemaFromCommitMetadata, syncMode private static Iterable syncModeAndSchemaFromCommitMetadata() { List opts = new ArrayList<>(); for (Object mode : SYNC_MODES) { @@ -92,9 +88,6 @@ private static Iterable syncModeAndSchemaFromCommitMetadata() { return opts; } - private HiveSyncTool hiveSyncTool; - private HoodieHiveClient hiveClient; - @AfterAll public static void cleanUpClass() { HiveTestUtil.shutdown(); @@ -132,100 +125,115 @@ public void teardown() throws Exception { @ParameterizedTest @MethodSource({"syncModeAndSchemaFromCommitMetadata"}) public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); - - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), + "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); + assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "No new partition should be added"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, newPartition); + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "New partition should be added"); // Update partitions - hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.updatePartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Partition count should remain the same"); - hiveClient.updatePartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.updatePartitionsToTable(hiveSyncConfig.tableName, newPartition); + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Partition count should remain the same"); // Alter partitions // Manually change a hive partition location to check if the sync will detect // it and generate a partition update event for it. - ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + ddlExecutor.runSQL("ALTER TABLE `" + hiveSyncConfig.tableName + "` PARTITION (`datestr`='2050-01-01') SET LOCATION '/some/new/location'"); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.empty()); + //writtenPartitionsSince.add(newPartition.get(0)); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.UPDATE, partitionEvents.iterator().next().eventType, "The one partition event must of type UPDATE"); - // Lets do the sync - reSyncHiveTable(); - + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); // Sync should update the changed partition to correct path - List tablePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List tablePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); assertEquals(6, tablePartitions.size(), "The one partition we wrote should be added to hive"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be 100"); } @ParameterizedTest @MethodSource({"syncMode"}) public void testSyncDataBase(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_DATABASE_NAME.key(), HiveTestUtil.DB_NAME); + hiveSyncConfig.databaseName = "database1"; // while autoCreateDatabase is false and database not exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); - reinitHiveSyncClient(); + hiveSyncConfig.autoCreateDatabase = false; // Lets do the sync - assertThrows(Exception.class, (this::reSyncHiveTable)); + assertThrows(Exception.class, () -> { + new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); + }); // while autoCreateDatabase is true and database not exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); - reinitHiveSyncClient(); - assertDoesNotThrow((this::reSyncHiveTable)); - assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), - "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); + hiveSyncConfig.autoCreateDatabase = true; + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertDoesNotThrow(() -> { + new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); + }); + assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), + "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); // while autoCreateDatabase is false and database exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "false"); - reinitHiveSyncClient(); - assertDoesNotThrow((this::reSyncHiveTable)); - assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), - "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); + hiveSyncConfig.autoCreateDatabase = false; + assertDoesNotThrow(() -> { + new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); + }); + assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), + "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); // while autoCreateDatabase is true and database exists; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_AUTO_CREATE_DATABASE.key(), "true"); - assertDoesNotThrow((this::reSyncHiveTable)); - assertTrue(hiveClient.doesDataBaseExist(HiveTestUtil.DB_NAME), - "DataBases " + HiveTestUtil.DB_NAME + " should exist after sync completes"); + hiveSyncConfig.autoCreateDatabase = true; + assertDoesNotThrow(() -> { + new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem).syncHoodieTable(); + }); + assertTrue(hiveClient.doesDataBaseExist(hiveSyncConfig.databaseName), + "DataBases " + hiveSyncConfig.databaseName + " should exist after sync completes"); } @ParameterizedTest @@ -233,9 +241,11 @@ public void testSyncDataBase(String syncMode) throws Exception { public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, boolean syncAsDataSourceTable, String syncMode) throws Exception { + HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; Map serdeProperties = new HashMap() { { - put("path", HiveTestUtil.basePath); + put("path", hiveSyncConfig.basePath); } }; @@ -245,20 +255,20 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.syncAsSparkDataSourceTable = syncAsDataSourceTable; + hiveSyncConfig.serdeProperties = ConfigUtils.configToString(serdeProperties); + hiveSyncConfig.tableProperties = ConfigUtils.configToString(tableProperties); String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - reinitHiveSyncClient(); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); SessionState.start(HiveTestUtil.getHiveConf()); Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); - String dbTableName = HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME; + String dbTableName = hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); List results = new ArrayList<>(); hiveDriver.getResults(results); @@ -269,10 +279,10 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, String sparkTableProperties = getSparkTableProperties(syncAsDataSourceTable, useSchemaFromCommitMetadata); assertEquals( "EXTERNAL\tTRUE\n" - + "last_commit_time_sync\t100\n" - + sparkTableProperties - + "tp_0\tp0\n" - + "tp_1\tp1", tblPropertiesWithoutDdlTime); + + "last_commit_time_sync\t100\n" + + sparkTableProperties + + "tp_0\tp0\n" + + "tp_1\tp1", tblPropertiesWithoutDdlTime); assertTrue(results.get(results.size() - 1).startsWith("transient_lastDdlTime")); results.clear(); @@ -280,7 +290,7 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, hiveDriver.run("SHOW CREATE TABLE " + dbTableName); hiveDriver.getResults(results); String ddl = String.join("\n", results); - assertTrue(ddl.contains("'path'='" + HiveTestUtil.basePath + "'")); + assertTrue(ddl.contains("'path'='" + hiveSyncConfig.basePath + "'")); if (syncAsDataSourceTable) { assertTrue(ddl.contains("'" + ConfigUtils.IS_QUERY_AS_RO_TABLE + "'='false'")); } @@ -289,33 +299,33 @@ public void testSyncCOWTableWithProperties(boolean useSchemaFromCommitMetadata, private String getSparkTableProperties(boolean syncAsDataSourceTable, boolean useSchemaFromCommitMetadata) { if (syncAsDataSourceTable) { if (useSchemaFromCommitMetadata) { - return "spark.sql.sources.provider\thudi\n" - + "spark.sql.sources.schema.numPartCols\t1\n" - + "spark.sql.sources.schema.numParts\t1\n" - + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":" - + "[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," - + "{\"name\":\"name\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"favorite_number\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," - + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" - + "spark.sql.sources.schema.partCol.0\tdatestr\n"; + return "spark.sql.sources.provider\thudi\n" + + "spark.sql.sources.schema.numPartCols\t1\n" + + "spark.sql.sources.schema.numParts\t1\n" + + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":" + + "[{\"name\":\"_hoodie_commit_time\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_commit_seqno\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_record_key\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_partition_path\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"_hoodie_file_name\",\"type\":\"string\",\"nullable\":true,\"metadata\":{}}," + + "{\"name\":\"name\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"favorite_number\",\"type\":\"integer\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}," + + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" + + "spark.sql.sources.schema.partCol.0\tdatestr\n"; } else { return "spark.sql.sources.provider\thudi\n" - + "spark.sql.sources.schema.numPartCols\t1\n" - + "spark.sql.sources.schema.numParts\t1\n" - + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":" - + "\"string\",\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_number\",\"type\":\"integer\"," - + "\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false," - + "\"metadata\":{}}]}\n" - + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" - + "spark.sql.sources.schema.partCol.0\tdatestr\n"; + + "spark.sql.sources.schema.numPartCols\t1\n" + + "spark.sql.sources.schema.numParts\t1\n" + + "spark.sql.sources.schema.part.0\t{\"type\":\"struct\",\"fields\":[{\"name\":\"name\",\"type\":" + + "\"string\",\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_number\",\"type\":\"integer\"," + + "\"nullable\":false,\"metadata\":{}},{\"name\":\"favorite_color\",\"type\":\"string\",\"nullable\":false," + + "\"metadata\":{}}]}\n" + + "{\"name\":\"datestr\",\"type\":\"string\",\"nullable\":false,\"metadata\":{}}]}\n" + + "spark.sql.sources.schema.partCol.0\tdatestr\n"; } } else { - return ""; + return ""; } } @@ -324,9 +334,11 @@ private String getSparkTableProperties(boolean syncAsDataSourceTable, boolean us public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, boolean syncAsDataSourceTable, String syncMode) throws Exception { + HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; Map serdeProperties = new HashMap() { { - put("path", HiveTestUtil.basePath); + put("path", hiveSyncConfig.basePath); } }; @@ -336,21 +348,20 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, put("tp_1", "p1"); } }; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_AS_DATA_SOURCE_TABLE.key(), String.valueOf(syncAsDataSourceTable)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_SERDE_PROPERTIES.key(), ConfigUtils.configToString(serdeProperties)); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_TABLE_PROPERTIES.key(), ConfigUtils.configToString(tableProperties)); - + hiveSyncConfig.syncAsSparkDataSourceTable = syncAsDataSourceTable; + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.serdeProperties = ConfigUtils.configToString(serdeProperties); + hiveSyncConfig.tableProperties = ConfigUtils.configToString(tableProperties); String instantTime = "100"; String deltaCommitTime = "101"; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - reinitHiveSyncClient(); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); - String roTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; - String rtTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String roTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; + String rtTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; String[] tableNames = new String[] {roTableName, rtTableName}; String[] readAsOptimizedResults = new String[] {"true", "false"}; @@ -359,8 +370,8 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); String sparkTableProperties = getSparkTableProperties(syncAsDataSourceTable, useSchemaFromCommitMetadata); - for (int i = 0; i < 2; i++) { - String dbTableName = HiveTestUtil.DB_NAME + "." + tableNames[i]; + for (int i = 0;i < 2; i++) { + String dbTableName = hiveSyncConfig.databaseName + "." + tableNames[i]; String readAsOptimized = readAsOptimizedResults[i]; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); @@ -371,10 +382,10 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, results.subList(0, results.size() - 1)); assertEquals( "EXTERNAL\tTRUE\n" - + "last_commit_time_sync\t101\n" - + sparkTableProperties - + "tp_0\tp0\n" - + "tp_1\tp1", tblPropertiesWithoutDdlTime); + + "last_commit_time_sync\t101\n" + + sparkTableProperties + + "tp_0\tp0\n" + + "tp_1\tp1", tblPropertiesWithoutDdlTime); assertTrue(results.get(results.size() - 1).startsWith("transient_lastDdlTime")); results.clear(); @@ -382,7 +393,7 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, hiveDriver.run("SHOW CREATE TABLE " + dbTableName); hiveDriver.getResults(results); String ddl = String.join("\n", results); - assertTrue(ddl.contains("'path'='" + HiveTestUtil.basePath + "'")); + assertTrue(ddl.contains("'path'='" + hiveSyncConfig.basePath + "'")); assertTrue(ddl.toLowerCase().contains("create external table")); if (syncAsDataSourceTable) { assertTrue(ddl.contains("'" + ConfigUtils.IS_QUERY_AS_RO_TABLE + "'='" + readAsOptimized + "'")); @@ -395,18 +406,19 @@ public void testSyncMORTableWithProperties(boolean useSchemaFromCommitMetadata, public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, boolean isManagedTable, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_CREATE_MANAGED_TABLE.key(), String.valueOf(isManagedTable)); + HiveSyncConfig hiveSyncConfig = HiveTestUtil.hiveSyncConfig; + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.createManagedTable = isManagedTable; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, useSchemaFromCommitMetadata); - reinitHiveSyncClient(); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); SessionState.start(HiveTestUtil.getHiveConf()); Driver hiveDriver = new org.apache.hadoop.hive.ql.Driver(HiveTestUtil.getHiveConf()); - String dbTableName = HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME; + String dbTableName = hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName; hiveDriver.run("SHOW TBLPROPERTIES " + dbTableName); List results = new ArrayList<>(); @@ -423,29 +435,37 @@ public void testSyncManagedTable(boolean useSchemaFromCommitMetadata, @ParameterizedTest @MethodSource("syncMode") public void testSyncWithSchema(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + + hiveSyncConfig.syncMode = syncMode; String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/complex.schema.avsc"); - reinitHiveSyncClient(); - reSyncHiveTable(); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); } @ParameterizedTest @MethodSource("syncMode") public void testSyncIncremental(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); - reinitHiveSyncClient(); - reSyncHiveTable(); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + // Lets do the sync + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Now lets create more partitions and these are the only ones which needs to be synced @@ -454,32 +474,37 @@ public void testSyncIncremental(String syncMode) throws Exception { HiveTestUtil.addCOWPartitions(1, true, true, dateTime, commitTime2); // Lets do the sync - reSyncHiveTable(); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(commitTime1)); assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit"); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD"); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); // Sync should add the one partition - reSyncHiveTable(); - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "The one partition we wrote should be added to hive"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be 101"); } @ParameterizedTest @MethodSource("syncMode") public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String commitTime1 = "100"; HiveTestUtil.createCOWTable(commitTime1, 5, true); - reinitHiveSyncClient(); - reSyncHiveTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + // Lets do the sync + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); - int fields = hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(); + int fields = hiveClient.getTableSchema(hiveSyncConfig.tableName).size(); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -487,30 +512,33 @@ public void testSyncIncrementalWithSchemaEvolution(String syncMode) throws Excep HiveTestUtil.addCOWPartitions(1, false, true, dateTime, commitTime2); // Lets do the sync - reinitHiveSyncClient(); - reSyncHiveTable(); - assertEquals(fields + 3, hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + + assertEquals(fields + 3, hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), "Hive Schema has evolved and should not be 3 more field"); - assertEquals("BIGINT", hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("favorite_number"), + assertEquals("BIGINT", hiveClient.getTableSchema(hiveSyncConfig.tableName).get("favorite_number"), "Hive Schema has evolved - Field favorite_number has evolved from int to long"); - assertTrue(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).containsKey("favorite_movie"), + assertTrue(hiveClient.getTableSchema(hiveSyncConfig.tableName).containsKey("favorite_movie"), "Hive Schema has evolved - Field favorite_movie was added"); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "The one partition we wrote should be added to hive"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be 101"); } @ParameterizedTest @MethodSource("syncMode") public void testUpdateTableComments(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test.avsc"); - reinitHiveSyncClient(); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); Map> alterCommentSchema = new HashMap<>(); //generate commented schema field @@ -526,9 +554,9 @@ public void testUpdateTableComments(String syncMode) throws Exception { } } - ddlExecutor.updateTableComments(HiveTestUtil.TABLE_NAME, alterCommentSchema); + ddlExecutor.updateTableComments(hiveSyncConfig.tableName,alterCommentSchema); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -541,14 +569,16 @@ public void testUpdateTableComments(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncWithCommentedSchema(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "false"); + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.syncComment = false; String commitTime = "100"; HiveTestUtil.createCOWTableWithSchema(commitTime, "/simple-test-doced.avsc"); - reinitHiveSyncClient(); - reSyncHiveTable(); - List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); int commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -557,10 +587,10 @@ public void testSyncWithCommentedSchema(String syncMode) throws Exception { } assertEquals(0, commentCnt, "hive schema field comment numbers should match the avro schema field doc numbers"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_COMMENT.key(), "true"); - reinitHiveSyncClient(); - reSyncHiveTable(); - fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(HiveTestUtil.TABLE_NAME); + hiveSyncConfig.syncComment = true; + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + fieldSchemas = hiveClient.getTableCommentUsingMetastoreClient(hiveSyncConfig.tableName); commentCnt = 0; for (FieldSchema fieldSchema : fieldSchemas) { if (!StringUtils.isNullOrEmpty(fieldSchema.getComment())) { @@ -573,29 +603,31 @@ public void testSyncWithCommentedSchema(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; String deltaCommitTime = "101"; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - String roTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(roTableName), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + String roTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE; + HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(roTableName), "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); assertTrue(hiveClient.doesTableExist(roTableName), "Table " + roTableName + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), + SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), "Hive Schema should match the table schema + partition field"); } @@ -613,18 +645,19 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync HiveTestUtil.addMORPartitions(1, true, false, useSchemaFromCommitMetadata, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - reinitHiveSyncClient(); - reSyncHiveTable(); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); if (useSchemaFromCommitMetadata) { assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. assertEquals(hiveClient.getTableSchema(roTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition @@ -637,38 +670,42 @@ public void testSyncMergeOnRead(boolean useSchemaFromCommitMetadata, String sync @ParameterizedTest @MethodSource("syncModeAndSchemaFromCommitMetadata") public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; String deltaCommitTime = "101"; - String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String snapshotTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; HiveTestUtil.createMORTable(instantTime, deltaCommitTime, 5, true, useSchemaFromCommitMetadata); - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(snapshotTableName), - "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + HoodieHiveClient hiveClientRT = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + + assertFalse(hiveClientRT.doesTableExist(snapshotTableName), + "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); - assertTrue(hiveClient.doesTableExist(snapshotTableName), - "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertTrue(hiveClientRT.doesTableExist(snapshotTableName), + "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should exist after sync completes"); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize(), + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), "Hive Schema should match the table schema + partition field"); } - assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), + assertEquals(5, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(deltaCommitTime, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(deltaCommitTime, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Now lets create more partitions and these are the only ones which needs to be synced @@ -679,52 +716,57 @@ public void testSyncMergeOnReadRT(boolean useSchemaFromCommitMetadata, String sy HiveTestUtil.addCOWPartitions(1, true, useSchemaFromCommitMetadata, dateTime, commitTime2); HiveTestUtil.addMORPartitions(1, true, false, useSchemaFromCommitMetadata, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - reinitHiveSyncClient(); - reSyncHiveTable(); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + hiveClientRT = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); if (useSchemaFromCommitMetadata) { - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); } else { // The data generated and schema in the data file do not have metadata columns, so we need a separate check. - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize(), + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size(), "Hive Schema should match the evolved table schema + partition field"); } // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), + assertEquals(6, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "The 2 partitions we wrote should be added to hive"); - assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); } @ParameterizedTest @MethodSource("syncMode") public void testMultiPartitionKeySync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), MultiPartKeysValueExtractor.class.getCanonicalName()); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year,month,day"); + HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(HiveTestUtil.hiveSyncConfig); + hiveSyncConfig.partitionValueExtractorClass = MultiPartKeysValueExtractor.class.getCanonicalName(); + hiveSyncConfig.tableName = "multi_part_key"; + hiveSyncConfig.partitionFields = Arrays.asList("year", "month", "day"); + HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); - HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); - - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(5, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(5, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // HoodieHiveClient had a bug where partition vals were sorted @@ -733,36 +775,41 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { String commitTime2 = "101"; HiveTestUtil.addCOWPartition("2010/01/02", true, true, commitTime2); - reinitHiveSyncClient(); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); List writtenPartitionsSince = hiveClient.getPartitionsWrittenToSince(Option.of(instantTime)); assertEquals(1, writtenPartitionsSince.size(), "We should have one partition written after 100 commit"); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); List partitionEvents = hiveClient.getPartitionEvents(hivePartitions, writtenPartitionsSince); assertEquals(1, partitionEvents.size(), "There should be only one partition event"); assertEquals(PartitionEventType.ADD, partitionEvents.iterator().next().eventType, "The one partition event must of type ADD"); - reSyncHiveTable(); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(6, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime2, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be 101"); // create partition "2010/02/01" and ensure sync works String commitTime3 = "102"; HiveTestUtil.addCOWPartition("2010/02/01", true, true, commitTime3); - HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); + HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - reinitHiveSyncClient(); - reSyncHiveTable(); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + + assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 3, "Hive Schema should match the table schema + partition fields"); - assertEquals(7, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(7, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(commitTime3, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); assertEquals(1, hiveClient.getPartitionsWrittenToSince(Option.of(commitTime2)).size()); } @@ -770,43 +817,49 @@ public void testMultiPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartitionKeySync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); - - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); - - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), + "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); // Adding of new partitions List newPartition = Arrays.asList("2050/01/01"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, Arrays.asList()); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, Arrays.asList()); + assertEquals(1, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "No new partition should be added"); - hiveClient.addPartitionsToTable(HiveTestUtil.TABLE_NAME, newPartition); - assertEquals(2, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + hiveClient.addPartitionsToTable(hiveSyncConfig.tableName, newPartition); + assertEquals(2, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "New partition should be added"); - reSyncHiveTable(); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); // Drop 1 partition. - ddlExecutor.runSQL("ALTER TABLE `" + HiveTestUtil.TABLE_NAME + ddlExecutor.runSQL("ALTER TABLE `" + hiveSyncConfig.tableName + "` DROP PARTITION (`datestr`='2050-01-01')"); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); assertEquals(1, hivePartitions.size(), "Table should have 1 partition because of the drop 1 partition"); } @@ -814,35 +867,42 @@ public void testDropPartitionKeySync(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testDropPartition(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 1, true); - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive + // session, then lead to connection retry, we can see there is a exception at log. + hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), + "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - List partitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + List partitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); assertEquals(1, partitions.size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), + assertEquals(instantTime, hiveClient.getLastCommitTimeSynced(hiveSyncConfig.tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); String partitiontoDelete = partitions.get(0).getValues().get(0).replace("-","/"); // create a replace commit to delete current partitions+ HiveTestUtil.createReplaceCommit("101", partitiontoDelete, WriteOperationType.DELETE_PARTITION, true, true); - // sync drop partitions - reinitHiveSyncClient(); - reSyncHiveTable(); + // sync drop partitins + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); - List hivePartitions = hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME); + hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + List hivePartitions = hiveClient.scanTablePartitions(hiveSyncConfig.tableName); assertEquals(0, hivePartitions.size(), "Table should have 0 partition because of the drop the only one partition"); } @@ -850,53 +910,62 @@ public void testDropPartition(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testNonPartitionedSync(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, true); - // Set partition value extractor to NonPartitionedExtractor - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), NonPartitionedExtractor.class.getCanonicalName()); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key(), "year, month, day"); - - HiveTestUtil.getCreatedTablesSet().add(HiveTestUtil.DB_NAME + "." + HiveTestUtil.TABLE_NAME); - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HiveSyncConfig hiveSyncConfig = HiveSyncConfig.copy(HiveTestUtil.hiveSyncConfig); + // Set partition value extractor to NonPartitionedExtractor + hiveSyncConfig.partitionValueExtractorClass = NonPartitionedExtractor.class.getCanonicalName(); + hiveSyncConfig.tableName = "non_partitioned"; + hiveSyncConfig.partitionFields = Arrays.asList("year", "month", "day"); + HiveTestUtil.getCreatedTablesSet().add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + + HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + assertTrue(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size(), "Hive Schema should match the table schema,ignoring the partition fields"); - assertEquals(0, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), + assertEquals(0, hiveClient.scanTablePartitions(hiveSyncConfig.tableName).size(), "Table should not have partitions because of the NonPartitionedExtractor"); } @ParameterizedTest @MethodSource("syncMode") public void testReadSchemaForMOR(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String commitTime = "100"; - String snapshotTableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; + String snapshotTableName = hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; HiveTestUtil.createMORTable(commitTime, "", 5, false, true); - reinitHiveSyncClient(); + HoodieHiveClient hiveClientRT = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - assertFalse(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertFalse(hiveClientRT.doesTableExist(snapshotTableName), "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should not exist initially"); // Lets do the sync - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); - assertTrue(hiveClient.doesTableExist(snapshotTableName), "Table " + HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + assertTrue(hiveClientRT.doesTableExist(snapshotTableName), "Table " + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE + " should exist after sync completes"); // Schema being read from compacted base files - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getSimpleSchema().getFields().size() + getPartitionFieldSize() + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getSimpleSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the table schema + partition field"); - assertEquals(5, hiveClient.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); + assertEquals(5, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "Table partitions should match the number of partitions we wrote"); // Now lets create more partitions and these are the only ones which needs to be synced ZonedDateTime dateTime = ZonedDateTime.now().plusDays(6); @@ -905,78 +974,84 @@ public void testReadSchemaForMOR(String syncMode) throws Exception { HiveTestUtil.addMORPartitions(1, true, false, true, dateTime, commitTime2, deltaCommitTime2); // Lets do the sync - reinitHiveSyncClient(); - reSyncHiveTable(); + tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + tool.syncHoodieTable(); + hiveClientRT = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); - // Schema being read from the log filesTestHiveSyncTool - assertEquals(hiveClient.getTableSchema(snapshotTableName).size(), - SchemaTestUtil.getEvolvedSchema().getFields().size() + getPartitionFieldSize() + // Schema being read from the log files + assertEquals(hiveClientRT.getTableSchema(snapshotTableName).size(), + SchemaTestUtil.getEvolvedSchema().getFields().size() + hiveSyncConfig.partitionFields.size() + HoodieRecord.HOODIE_META_COLUMNS.size(), "Hive Schema should match the evolved table schema + partition field"); // Sync should add the one partition - assertEquals(6, hiveClient.scanTablePartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); - assertEquals(deltaCommitTime2, hiveClient.getLastCommitTimeSynced(snapshotTableName).get(), + assertEquals(6, hiveClientRT.scanTablePartitions(snapshotTableName).size(), "The 1 partition we wrote should be added to hive"); + assertEquals(deltaCommitTime2, hiveClientRT.getLastCommitTimeSynced(snapshotTableName).get(), "The last commit that was synced should be 103"); } @Test public void testConnectExceptionIgnoreConfigSet() throws IOException, URISyntaxException, HiveException, MetaException { + hiveSyncConfig.useJdbc = true; + HiveTestUtil.hiveSyncConfig.useJdbc = true; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; String instantTime = "100"; HiveTestUtil.createCOWTable(instantTime, 5, false); - reinitHiveSyncClient(); - HoodieHiveClient prevHiveClient = hiveClient; - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); - + HoodieHiveClient hiveClient = + new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); // Lets do the sync - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_IGNORE_EXCEPTIONS.key(), "true"); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_URL.key(), hiveSyncProps.getString(HiveSyncConfig.HIVE_URL.key()) - .replace(String.valueOf(HiveTestUtil.hiveTestService.getHiveServerPort()), String.valueOf(NetworkTestUtils.nextFreePort()))); - reinitHiveSyncClient(); - reSyncHiveTable(); - - assertNull(hiveClient); - assertFalse(prevHiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), - "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + + HiveSyncConfig syncToolConfig = HiveSyncConfig.copy(hiveSyncConfig); + syncToolConfig.ignoreExceptions = true; + syncToolConfig.jdbcUrl = HiveTestUtil.hiveSyncConfig.jdbcUrl + .replace(String.valueOf(HiveTestUtil.hiveTestService.getHiveServerPort()), String.valueOf(NetworkTestUtils.nextFreePort())); + HiveSyncTool tool = new HiveSyncTool(syncToolConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + + assertFalse(hiveClient.doesTableExist(hiveSyncConfig.tableName), + "Table " + hiveSyncConfig.tableName + " should not exist initially"); } private void verifyOldParquetFileTest(HoodieHiveClient hiveClient, String emptyCommitTime) throws Exception { - assertTrue(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should exist after sync completes"); - assertEquals(hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), + assertTrue(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should exist after sync completes"); + assertEquals(hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(), hiveClient.getDataSchema().getColumns().size() + 1, "Hive Schema should match the table schema + partition field"); - assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.TABLE_NAME).size(), "Table partitions should match the number of partitions we wrote"); + assertEquals(1, hiveClient.scanTablePartitions(HiveTestUtil.hiveSyncConfig.tableName).size(),"Table partitions should match the number of partitions we wrote"); assertEquals(emptyCommitTime, - hiveClient.getLastCommitTimeSynced(HiveTestUtil.TABLE_NAME).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); + hiveClient.getLastCommitTimeSynced(HiveTestUtil.hiveSyncConfig.tableName).get(),"The last commit that was synced should be updated in the TBLPROPERTIES"); // make sure correct schema is picked Schema schema = SchemaTestUtil.getSimpleSchema(); for (Field field : schema.getFields()) { assertEquals(field.schema().getType().getName(), - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get(field.name()).toLowerCase(), + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get(field.name()).toLowerCase(), String.format("Hive Schema Field %s was added", field)); } assertEquals("string", - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).get("datestr").toLowerCase(), "Hive Schema Field datestr was added"); assertEquals(schema.getFields().size() + 1 + HoodieRecord.HOODIE_META_COLUMNS.size(), - hiveClient.getTableSchema(HiveTestUtil.TABLE_NAME).size(), "Hive Schema fields size"); + hiveClient.getTableSchema(HiveTestUtil.hiveSyncConfig.tableName).size(),"Hive Schema fields size"); } @ParameterizedTest @MethodSource("syncMode") public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); // create empty commit final String emptyCommitTime = "200"; HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); - reinitHiveSyncClient(); - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + assertFalse(hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); - reinitHiveSyncClient(); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); verifyOldParquetFileTest(hiveClient, emptyCommitTime); } @@ -984,7 +1059,8 @@ public void testPickingOlderParquetFileIfLatestIsEmptyCommit(String syncMode) th @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -996,15 +1072,17 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM // create empty commit final String emptyCommitTime = "200"; - HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime, basePath); + HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime, hiveSyncConfig.basePath); - reinitHiveSyncClient(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); assertFalse( - hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); - HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, getHiveConf(), fileSystem); // now delete the evolved commit instant - Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + hiveClient.getActiveTimeline().getInstants() .filter(inst -> inst.getTimestamp().equals(commitTime2)) .findFirst().get().getFileName()); @@ -1017,13 +1095,15 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFails(String syncM } // table should not be synced yet - assertFalse(hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist at all"); + assertFalse( + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName),"Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist at all"); } @ParameterizedTest @MethodSource("syncMode") public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTable(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; final String commitTime = "100"; HiveTestUtil.createCOWTable(commitTime, 1, true); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); @@ -1031,11 +1111,13 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa final String emptyCommitTime = "200"; HiveTestUtil.createCommitFileWithSchema(commitMetadata, emptyCommitTime, true); //HiveTestUtil.createCommitFile(commitMetadata, emptyCommitTime); - reinitHiveSyncClient(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); assertFalse( - hiveClient.doesTableExist(HiveTestUtil.TABLE_NAME), "Table " + HiveTestUtil.TABLE_NAME + " should not exist initially"); + hiveClient.doesTableExist(HiveTestUtil.hiveSyncConfig.tableName), "Table " + HiveTestUtil.hiveSyncConfig.tableName + " should not exist initially"); - reSyncHiveTable(); + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); verifyOldParquetFileTest(hiveClient, emptyCommitTime); @@ -1046,19 +1128,18 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa //HiveTestUtil.createCommitFileWithSchema(commitMetadata, "400", false); // create another empty commit //HiveTestUtil.createCommitFile(commitMetadata, "400"); // create another empty commit - reinitHiveSyncClient(); + tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + HoodieHiveClient hiveClientLatest = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); // now delete the evolved commit instant - Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + hiveClient.getActiveTimeline().getInstants() - .filter(inst -> inst.getTimestamp().equals(commitTime2)) - .findFirst().get().getFileName()); + Path fullPath = new Path(HiveTestUtil.hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + + hiveClientLatest.getActiveTimeline().getInstants() + .filter(inst -> inst.getTimestamp().equals(commitTime2)) + .findFirst().get().getFileName()); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); try { - reSyncHiveTable(); + tool.syncHoodieTable(); } catch (RuntimeException e) { // we expect the table sync to fail - } finally { - reinitHiveSyncClient(); } // old sync values should be left intact @@ -1068,13 +1149,15 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa @ParameterizedTest @MethodSource("syncMode") public void testTypeConverter(String syncMode) throws Exception { - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); + hiveSyncConfig.syncMode = syncMode; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; HiveTestUtil.createCOWTable("100", 5, true); // create database. - ddlExecutor.runSQL("create database " + HiveTestUtil.DB_NAME); - reinitHiveSyncClient(); - String tableName = HiveTestUtil.TABLE_NAME; - String tableAbsoluteName = String.format(" `%s.%s` ", HiveTestUtil.DB_NAME, tableName); + ddlExecutor.runSQL("create database " + hiveSyncConfig.databaseName); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + String tableName = HiveTestUtil.hiveSyncConfig.tableName; + String tableAbsoluteName = String.format(" `%s.%s` ", HiveTestUtil.hiveSyncConfig.databaseName, tableName); String dropTableSql = String.format("DROP TABLE IF EXISTS %s ", tableAbsoluteName); String createTableSqlPrefix = String.format("CREATE TABLE IF NOT EXISTS %s ", tableAbsoluteName); String errorMsg = "An error occurred in decimal type converting."; @@ -1108,40 +1191,31 @@ public void testTypeConverter(String syncMode) throws Exception { @ParameterizedTest @MethodSource("syncMode") public void testSyncWithoutDiffs(String syncMode) throws Exception { - String tableName = HiveTestUtil.TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), syncMode); - hiveSyncProps.setProperty(HiveSyncConfig.META_SYNC_CONDITIONAL_SYNC.key(), "true"); + hiveSyncConfig.syncMode = syncMode; + hiveSyncConfig.isConditionalSync = true; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 2; + String tableName = HiveTestUtil.hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE; String commitTime0 = "100"; String commitTime1 = "101"; String commitTime2 = "102"; HiveTestUtil.createMORTable(commitTime0, commitTime1, 2, true, true); - reinitHiveSyncClient(); - reSyncHiveTable(); + HoodieHiveClient hiveClient = + new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + + HiveSyncTool tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); assertTrue(hiveClient.doesTableExist(tableName)); assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); HiveTestUtil.addMORPartitions(0, true, true, true, ZonedDateTime.now().plusDays(2), commitTime1, commitTime2); - reSyncHiveTable(); + tool = new HiveSyncTool(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); + tool.syncHoodieTable(); + hiveClient = new HoodieHiveClient(HiveTestUtil.hiveSyncConfig, HiveTestUtil.getHiveConf(), HiveTestUtil.fileSystem); assertEquals(commitTime1, hiveClient.getLastCommitTimeSynced(tableName).get()); } - private void reSyncHiveTable() { - hiveSyncTool.syncHoodieTable(); - // we need renew the hiveclient after tool.syncHoodieTable(), because it will close hive - // session, then lead to connection retry, we can see there is a exception at log. - reinitHiveSyncClient(); - } - - private void reinitHiveSyncClient() { - hiveSyncTool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); - hiveClient = hiveSyncTool.hoodieHiveClient; - } - - private int getPartitionFieldSize() { - return hiveSyncProps.getString(HiveSyncConfig.META_SYNC_PARTITION_FIELDS.key()).split(",").length; - } } diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 3cdbe0d8bb757..e66bb7c914645 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -22,7 +22,6 @@ 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; @@ -76,6 +75,7 @@ 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,21 +89,16 @@ @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; - private static HiveSyncConfig hiveSyncConfig; + public static HiveSyncConfig hiveSyncConfig; private static DateTimeFormatter dtfOut; + public static FileSystem fileSystem; private static Set createdTablesSet = new HashSet<>(); + public static QueryBasedDDLExecutor ddlExecutor; public static void setUp() throws IOException, InterruptedException, HiveException, MetaException { configuration = new Configuration(); @@ -117,21 +112,16 @@ public static void setUp() throws IOException, InterruptedException, HiveExcepti } fileSystem = FileSystem.get(configuration); - 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); + 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"); dtfOut = DateTimeFormatter.ofPattern("yyyy/MM/dd"); ddlExecutor = new HiveQueryDDLExecutor(hiveSyncConfig, fileSystem, getHiveConf()); @@ -148,18 +138,18 @@ public static void clearIncrementalPullSetup(String path1, String path2) throws } public static void clear() throws IOException, HiveException, MetaException { - fileSystem.delete(new Path(basePath), true); + fileSystem.delete(new Path(hiveSyncConfig.basePath), true); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(TABLE_NAME) + .setTableName(hiveSyncConfig.tableName) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, basePath); + .initTable(configuration, hiveSyncConfig.basePath); for (String tableName : createdTablesSet) { ddlExecutor.runSQL("drop table if exists " + tableName); } createdTablesSet.clear(); - ddlExecutor.runSQL("drop database if exists " + DB_NAME + " cascade"); + ddlExecutor.runSQL("drop database if exists " + hiveSyncConfig.databaseName + " cascade"); } public static HiveConf getHiveConf() { @@ -199,7 +189,7 @@ public static void createCOWTable(String instantTime, int numberOfPartitions, bo public static void createCOWTable(String instantTime, int numberOfPartitions, boolean useSchemaFromCommitMetadata) throws IOException, URISyntaxException { - createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, basePath, DB_NAME, TABLE_NAME); + createCOWTable(instantTime, numberOfPartitions, useSchemaFromCommitMetadata, hiveSyncConfig.basePath, hiveSyncConfig.databaseName, hiveSyncConfig.tableName); } public static void createReplaceCommit(String instantTime, String partitions, WriteOperationType type, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata) @@ -215,13 +205,13 @@ public static void createReplaceCommit(String instantTime, String partitions, Wr public static void createCOWTableWithSchema(String instantTime, String schemaFileName) throws IOException, URISyntaxException { - Path path = new Path(basePath); - FileIOUtils.deleteDirectory(new File(basePath)); + Path path = new Path(hiveSyncConfig.basePath); + FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.COPY_ON_WRITE) - .setTableName(TABLE_NAME) + .setTableName(hiveSyncConfig.tableName) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, basePath); + .initTable(configuration, hiveSyncConfig.basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); @@ -229,7 +219,7 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); String partitionPath = dateTime.format(dtfOut); - Path partPath = new Path(basePath + "/" + partitionPath); + Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath); fileSystem.makeQualified(partPath); fileSystem.mkdirs(partPath); List writeStats = new ArrayList<>(); @@ -243,30 +233,30 @@ public static void createCOWTableWithSchema(String instantTime, String schemaFil writeStats.add(writeStat); writeStats.forEach(s -> commitMetadata.addWriteStat(partitionPath, s)); commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schema.toString()); - createdTablesSet.add(DB_NAME + "." + TABLE_NAME); - createCommitFile(commitMetadata, instantTime, basePath); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath); } public static void createMORTable(String commitTime, String deltaCommitTime, int numberOfPartitions, boolean createDeltaCommit, boolean useSchemaFromCommitMetadata) throws IOException, URISyntaxException, InterruptedException { - Path path = new Path(basePath); - FileIOUtils.deleteDirectory(new File(basePath)); + Path path = new Path(hiveSyncConfig.basePath); + FileIOUtils.deleteDirectory(new File(hiveSyncConfig.basePath)); HoodieTableMetaClient.withPropertyBuilder() .setTableType(HoodieTableType.MERGE_ON_READ) - .setTableName(TABLE_NAME) + .setTableName(hiveSyncConfig.tableName) .setPayloadClass(HoodieAvroPayload.class) - .initTable(configuration, basePath); + .initTable(configuration, hiveSyncConfig.basePath); boolean result = fileSystem.mkdirs(path); checkResult(result); ZonedDateTime dateTime = ZonedDateTime.now(); HoodieCommitMetadata commitMetadata = createPartitions(numberOfPartitions, true, - useSchemaFromCommitMetadata, dateTime, commitTime, basePath); + useSchemaFromCommitMetadata, dateTime, commitTime, hiveSyncConfig.basePath); createdTablesSet - .add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); + .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); createdTablesSet - .add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); + .add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -284,26 +274,26 @@ public static void createMORTable(String commitTime, String deltaCommitTime, int 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, basePath); - createdTablesSet.add(DB_NAME + "." + TABLE_NAME); - createCommitFile(commitMetadata, instantTime, basePath); + createPartitions(numberOfPartitions, isParquetSchemaSimple, useSchemaFromCommitMetadata, startFrom, instantTime, hiveSyncConfig.basePath); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + createCommitFile(commitMetadata, instantTime, hiveSyncConfig.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(DB_NAME + "." + TABLE_NAME); - createCommitFile(commitMetadata, instantTime, basePath); + createdTablesSet.add(hiveSyncConfig.databaseName + "." + hiveSyncConfig.tableName); + createCommitFile(commitMetadata, instantTime, hiveSyncConfig.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, basePath); - createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_READ_OPTIMIZED_TABLE); - createdTablesSet.add(DB_NAME + "." + TABLE_NAME + HiveSyncTool.SUFFIX_SNAPSHOT_TABLE); + 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); HoodieCommitMetadata compactionMetadata = new HoodieCommitMetadata(); commitMetadata.getPartitionToWriteStats() .forEach((key, value) -> value.forEach(l -> compactionMetadata.addWriteStat(key, l))); @@ -356,7 +346,7 @@ private static HoodieCommitMetadata createPartitions(int numberOfPartitions, boo private static HoodieCommitMetadata createPartition(String partitionPath, boolean isParquetSchemaSimple, boolean useSchemaFromCommitMetadata, String instantTime) throws IOException, URISyntaxException { HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - Path partPath = new Path(basePath + "/" + partitionPath); + Path partPath = new Path(hiveSyncConfig.basePath + "/" + partitionPath); fileSystem.makeQualified(partPath); fileSystem.mkdirs(partPath); List writeStats = createTestData(partPath, isParquetSchemaSimple, instantTime); @@ -481,7 +471,7 @@ public static void createCommitFile(HoodieCommitMetadata commitMetadata, String public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeReplaceFileName(instantTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); @@ -490,13 +480,13 @@ public static void createReplaceCommitFile(HoodieCommitMetadata commitMetadata, public static void createCommitFileWithSchema(HoodieCommitMetadata commitMetadata, String instantTime, boolean isSimpleSchema) throws IOException { addSchemaToCommitMetadata(commitMetadata, isSimpleSchema, true); - createCommitFile(commitMetadata, instantTime, basePath); + createCommitFile(commitMetadata, instantTime, hiveSyncConfig.basePath); } private static void createCompactionCommitFile(HoodieCommitMetadata commitMetadata, String instantTime) throws IOException { byte[] bytes = commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeCommitFileName(instantTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); @@ -506,7 +496,7 @@ private static void createCompactionCommitFile(HoodieCommitMetadata commitMetada private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime) throws IOException { byte[] bytes = deltaCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8); - Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + Path fullPath = new Path(hiveSyncConfig.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeDeltaFileName(deltaCommitTime)); FSDataOutputStream fsout = fileSystem.create(fullPath, true); fsout.write(bytes); diff --git a/hudi-sync/hudi-sync-common/pom.xml b/hudi-sync/hudi-sync-common/pom.xml index 19a83e757c0d4..1f1abb4f177f1 100644 --- a/hudi-sync/hudi-sync-common/pom.xml +++ b/hudi-sync/hudi-sync-common/pom.xml @@ -45,65 +45,6 @@ org.apache.hadoop hadoop-common - - com.beust - jcommander - - - - - org.apache.logging.log4j - log4j-core - test - - - - org.junit.jupiter - junit-jupiter-api - test - - - - org.junit.jupiter - junit-jupiter-engine - test - - - - org.junit.vintage - junit-vintage-engine - test - - - - org.junit.jupiter - junit-jupiter-params - test - - - - org.mockito - mockito-junit-jupiter - test - - - - org.junit.platform - junit-platform-runner - test - - - - org.junit.platform - junit-platform-suite-api - test - - - - org.junit.platform - junit-platform-commons - test - diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java index 680b4a17ef5d9..6621468ee2e25 100644 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java +++ b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/AbstractSyncTool.java @@ -17,31 +17,17 @@ 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 final Configuration conf; - protected final FileSystem fs; - protected TypedProperties props; - - public AbstractSyncTool(TypedProperties props, Configuration conf, FileSystem fs) { - this.props = props; - this.conf = conf; - this.fs = fs; - } + protected Properties props; + protected FileSystem fileSystem; - @Deprecated public AbstractSyncTool(Properties props, FileSystem fileSystem) { - this(new TypedProperties(props), fileSystem.getConf(), fileSystem); + this.props = props; + this.fileSystem = fileSystem; } public abstract void syncHoodieTable(); diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java deleted file mode 100644 index dc2b21ba4504f..0000000000000 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/HoodieSyncConfig.java +++ /dev/null @@ -1,188 +0,0 @@ -/* - * 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 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 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 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 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> 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 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 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> 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 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> 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 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 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 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 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 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); - } -} diff --git a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java deleted file mode 100644 index f40e53b80f871..0000000000000 --- a/hudi-sync/hudi-sync-common/src/main/java/org/apache/hudi/sync/common/util/SyncUtilHelpers.java +++ /dev/null @@ -1,72 +0,0 @@ -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); - } - } - } -} diff --git a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java b/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java deleted file mode 100644 index dc9dee8b42ea4..0000000000000 --- a/hudi-sync/hudi-sync-common/src/test/java/org/apache/hudi/sync/common/util/TestSyncUtilHelpers.java +++ /dev/null @@ -1,124 +0,0 @@ -/* - * 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) { - } - } -} diff --git a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties deleted file mode 100644 index b21b5d4070c41..0000000000000 --- a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire-quiet.properties +++ /dev/null @@ -1,29 +0,0 @@ -### -# 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 diff --git a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties b/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties deleted file mode 100644 index c03e808cca1f8..0000000000000 --- a/hudi-sync/hudi-sync-common/src/test/resources/log4j-surefire.properties +++ /dev/null @@ -1,30 +0,0 @@ -### -# 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 \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java index 84b7933767610..833fce295e326 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/BootstrapExecutor.java @@ -18,6 +18,7 @@ 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; @@ -33,13 +34,13 @@ 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; @@ -160,16 +161,12 @@ public void execute() throws IOException { */ private void syncHive() { if (cfg.enableHiveSync || cfg.enableMetaSync) { - 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(); + 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(); } } @@ -198,4 +195,4 @@ private void initializeTable() throws IOException { public HoodieWriteConfig getBootstrapConfig() { return bootstrapConfig; } -} +} \ No newline at end of file diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java index 50338e5510c5b..7dd9a36892cf2 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/DeltaSync.java @@ -42,12 +42,12 @@ 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.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory; import org.apache.hudi.metrics.HoodieMetrics; -import org.apache.hudi.sync.common.util.SyncUtilHelpers; +import org.apache.hudi.sync.common.AbstractSyncTool; import org.apache.hudi.utilities.UtilHelpers; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallback; import org.apache.hudi.utilities.callback.kafka.HoodieWriteCommitKafkaCallbackConfig; @@ -81,6 +81,7 @@ 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; @@ -98,6 +99,7 @@ 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; @@ -690,24 +692,44 @@ private void syncMeta(HoodieDeltaStreamerMetrics metrics) { 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(); - SyncUtilHelpers.runHoodieMetaSync(impl.trim(), metaProps, conf, fs, cfg.targetBasePath, cfg.baseFileFormat); + 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(); + } 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 diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java index 3c72eb58af1e8..bcd7b3b7d8ac6 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/deltastreamer/HoodieMultiTableDeltaStreamer.java @@ -19,6 +19,7 @@ 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; @@ -27,7 +28,6 @@ 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 @@ private void populateTableExecutionContextList(TypedProperties properties, Strin 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(HoodieSyncConfig.META_SYNC_TABLE_NAME.key(), ""))) { + if (cfg.enableMetaSync && StringUtils.isNullOrEmpty(tableProperties.getString(DataSourceWriteOptions.HIVE_TABLE().key(), ""))) { throw new HoodieException("Meta sync table field not provided!"); } populateSchemaProviderProps(cfg, tableProperties); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java index f6ea5c0f55485..a2f70e0416942 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHiveIncrementalPuller.java @@ -20,13 +20,10 @@ 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; @@ -41,7 +38,7 @@ import java.time.Instant; import static org.apache.hudi.hive.testutils.HiveTestUtil.fileSystem; -import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncProps; +import static org.apache.hudi.hive.testutils.HiveTestUtil.hiveSyncConfig; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -72,12 +69,12 @@ public void testInitHiveIncrementalPuller() { } private HiveIncrementalPuller.Config getHivePullerConfig(String incrementalSql) throws IOException { - 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.hiveJDBCUrl = hiveSyncConfig.jdbcUrl; + config.hiveUsername = hiveSyncConfig.hiveUser; + config.hivePassword = hiveSyncConfig.hivePass; config.hoodieTmpDir = Files.createTempDirectory("hivePullerTest").toUri().toString(); - config.sourceDb = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key()); - config.sourceTable = hiveSyncProps.getString(HoodieSyncConfig.META_SYNC_TABLE_NAME.key()); + config.sourceDb = hiveSyncConfig.databaseName; + config.sourceTable = hiveSyncConfig.tableName; config.targetDb = "tgtdb"; config.targetTable = "test2"; config.tmpDb = "tmp_db"; @@ -101,9 +98,9 @@ private void createIncrementalSqlFile(String text, HiveIncrementalPuller.Config private void createSourceTable() throws IOException, URISyntaxException { String instantTime = "101"; HiveTestUtil.createCOWTable(instantTime, 5, true); - hiveSyncProps.setProperty(HiveSyncConfig.HIVE_SYNC_MODE.key(), "jdbc"); - - HiveSyncTool tool = new HiveSyncTool(hiveSyncProps, HiveTestUtil.getHiveConf(), fileSystem); + hiveSyncConfig.syncMode = "jdbc"; + HiveTestUtil.hiveSyncConfig.batchSyncNum = 3; + HiveSyncTool tool = new HiveSyncTool(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); tool.syncHoodieTable(); } @@ -116,20 +113,20 @@ private void createTargetTable() throws IOException, URISyntaxException { tool.syncHoodieTable(); } - 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 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 getAssertionSyncConfig(String databaseName) { - TypedProperties assertHiveSyncProps = new TypedProperties(hiveSyncProps); - assertHiveSyncProps.setProperty(HoodieSyncConfig.META_SYNC_DATABASE_NAME.key(), databaseName); - return assertHiveSyncProps; + private HiveSyncConfig getAssertionSyncConfig(String databaseName) { + HiveSyncConfig config = HiveSyncConfig.copy(hiveSyncConfig); + config.databaseName = databaseName; + return config; } private void createTables() throws IOException, URISyntaxException { @@ -161,11 +158,12 @@ public void testPullerWithoutSourceInSql() throws IOException, URISyntaxExceptio 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(new HiveSyncConfig(hiveSyncProps), HiveTestUtil.getHiveConf(), fileSystem); + HoodieHiveClient hiveClient = new HoodieHiveClient(hiveSyncConfig, HiveTestUtil.getHiveConf(), fileSystem); hiveClient.createDatabase(cfg.tmpDb); HiveIncrementalPuller puller = new HiveIncrementalPuller(cfg); puller.saveDelta(); - HoodieHiveClient assertingClient = new HoodieHiveClient(new HiveSyncConfig(getAssertionSyncConfig(cfg.tmpDb)), HiveTestUtil.getHiveConf(), fileSystem); + HiveSyncConfig assertingConfig = getAssertionSyncConfig(cfg.tmpDb); + HoodieHiveClient assertingClient = new HoodieHiveClient(assertingConfig, HiveTestUtil.getHiveConf(), fileSystem); String tmpTable = cfg.targetTable + "__" + cfg.sourceTable; assertTrue(assertingClient.doesTableExist(tmpTable)); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java index d495721147b55..2a32e3aeca327 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/HoodieDeltaStreamerTestBase.java @@ -18,6 +18,7 @@ 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; @@ -27,7 +28,6 @@ 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; @@ -183,11 +183,11 @@ protected static void writeCommonPropsToFile(FileSystem dfs, String dfsBasePath) props.setProperty("hoodie.deltastreamer.schemaprovider.target.schema.file", dfsBasePath + "/target.avsc"); // Hive Configs - 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(), + 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(), MultiPartKeysValueExtractor.class.getName()); UtilitiesTestBase.Helpers.savePropsToDFS(props, dfs, dfsBasePath + "/" + PROPS_FILENAME_TEST_SOURCE); } @@ -245,11 +245,11 @@ protected static void populateCommonKafkaProps(TypedProperties props, String bro protected static void populateCommonHiveProps(TypedProperties props) { // Hive Configs - 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(), + 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(), MultiPartKeysValueExtractor.class.getName()); }