[HUDI-2883] Refactor hive sync tool / config to use reflection and standardize configs (#4175)
- Refactor hive sync tool / config to use reflection and standardize configs Co-authored-by: sivabalan <n.siva.b@gmail.com> Co-authored-by: Rajesh Mahindra <rmahindra@Rajeshs-MacBook-Pro.local> Co-authored-by: Raymond Xu <2701446+xushiyan@users.noreply.github.com>
This commit is contained in:
@@ -32,8 +32,6 @@ import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.connect.ControlMessage;
|
||||
import org.apache.hudi.connect.writers.KafkaConnectConfigs;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor;
|
||||
import org.apache.hudi.keygen.BaseKeyGenerator;
|
||||
import org.apache.hudi.keygen.CustomAvroKeyGenerator;
|
||||
import org.apache.hudi.keygen.CustomKeyGenerator;
|
||||
@@ -59,7 +57,6 @@ import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.Arrays;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
@@ -269,32 +266,4 @@ public class KafkaConnectUtils {
|
||||
ControlMessage.ConnectWriteStatus connectWriteStatus = participantInfo.getWriteStatus();
|
||||
return SerializationUtils.deserialize(connectWriteStatus.getSerializedWriteStatus().toByteArray());
|
||||
}
|
||||
|
||||
/**
|
||||
* Build Hive Sync Config
|
||||
* Note: This method is a temporary solution.
|
||||
* Future solutions can be referred to: https://issues.apache.org/jira/browse/HUDI-3199
|
||||
*/
|
||||
public static HiveSyncConfig buildSyncConfig(TypedProperties props, String tableBasePath) {
|
||||
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
|
||||
hiveSyncConfig.basePath = tableBasePath;
|
||||
hiveSyncConfig.usePreApacheInputFormat = props.getBoolean(KafkaConnectConfigs.HIVE_USE_PRE_APACHE_INPUT_FORMAT, false);
|
||||
hiveSyncConfig.databaseName = props.getString(KafkaConnectConfigs.HIVE_DATABASE, "default");
|
||||
hiveSyncConfig.tableName = props.getString(KafkaConnectConfigs.HIVE_TABLE, "");
|
||||
hiveSyncConfig.hiveUser = props.getString(KafkaConnectConfigs.HIVE_USER, "");
|
||||
hiveSyncConfig.hivePass = props.getString(KafkaConnectConfigs.HIVE_PASS, "");
|
||||
hiveSyncConfig.jdbcUrl = props.getString(KafkaConnectConfigs.HIVE_URL, "");
|
||||
hiveSyncConfig.partitionFields = props.getStringList(KafkaConnectConfigs.HIVE_PARTITION_FIELDS, ",", Collections.emptyList());
|
||||
hiveSyncConfig.partitionValueExtractorClass =
|
||||
props.getString(KafkaConnectConfigs.HIVE_PARTITION_EXTRACTOR_CLASS, SlashEncodedDayPartitionValueExtractor.class.getName());
|
||||
hiveSyncConfig.useJdbc = props.getBoolean(KafkaConnectConfigs.HIVE_USE_JDBC, true);
|
||||
if (props.containsKey(KafkaConnectConfigs.HIVE_SYNC_MODE)) {
|
||||
hiveSyncConfig.syncMode = props.getString(KafkaConnectConfigs.HIVE_SYNC_MODE);
|
||||
}
|
||||
hiveSyncConfig.autoCreateDatabase = props.getBoolean(KafkaConnectConfigs.HIVE_AUTO_CREATE_DATABASE, true);
|
||||
hiveSyncConfig.ignoreExceptions = props.getBoolean(KafkaConnectConfigs.HIVE_IGNORE_EXCEPTIONS, false);
|
||||
hiveSyncConfig.skipROSuffix = props.getBoolean(KafkaConnectConfigs.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE, false);
|
||||
hiveSyncConfig.supportTimestamp = props.getBoolean(KafkaConnectConfigs.HIVE_SUPPORT_TIMESTAMP_TYPE, false);
|
||||
return hiveSyncConfig;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -30,22 +30,17 @@ import org.apache.hudi.common.model.HoodieCommitMetadata;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.common.util.StringUtils;
|
||||
import org.apache.hudi.config.HoodieWriteConfig;
|
||||
import org.apache.hudi.connect.transaction.TransactionCoordinator;
|
||||
import org.apache.hudi.connect.utils.KafkaConnectUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.hive.HiveSyncConfig;
|
||||
import org.apache.hudi.hive.HiveSyncTool;
|
||||
import org.apache.hudi.hive.ddl.HiveSyncMode;
|
||||
import org.apache.hudi.keygen.KeyGenerator;
|
||||
import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory;
|
||||
import org.apache.hudi.sync.common.AbstractSyncTool;
|
||||
import org.apache.hudi.sync.common.HoodieSyncConfig;
|
||||
import org.apache.hudi.sync.common.util.SyncUtilHelpers;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.hive.conf.HiveConf;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -54,7 +49,6 @@ import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
@@ -167,43 +161,10 @@ public class KafkaConnectTransactionServices implements ConnectTransactionServic
|
||||
if (connectConfigs.isMetaSyncEnabled()) {
|
||||
Set<String> syncClientToolClasses = new HashSet<>(
|
||||
Arrays.asList(connectConfigs.getMetaSyncClasses().split(",")));
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
|
||||
for (String impl : syncClientToolClasses) {
|
||||
impl = impl.trim();
|
||||
switch (impl) {
|
||||
case "org.apache.hudi.hive.HiveSyncTool":
|
||||
syncHive();
|
||||
break;
|
||||
default:
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, new Configuration());
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(connectConfigs.getProps());
|
||||
properties.put("basePath", tableBasePath);
|
||||
AbstractSyncTool syncTool = (AbstractSyncTool) ReflectionUtils.loadClass(impl, new Class[] {Properties.class, FileSystem.class}, properties, fs);
|
||||
syncTool.syncHoodieTable();
|
||||
}
|
||||
SyncUtilHelpers.runHoodieMetaSync(impl.trim(), connectConfigs.getProps(), hadoopConf, fs, tableBasePath, HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT.defaultValue());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void syncHive() {
|
||||
HiveSyncConfig hiveSyncConfig = KafkaConnectUtils.buildSyncConfig(new TypedProperties(connectConfigs.getProps()), tableBasePath);
|
||||
String url;
|
||||
if (!StringUtils.isNullOrEmpty(hiveSyncConfig.syncMode) && HiveSyncMode.of(hiveSyncConfig.syncMode) == HiveSyncMode.HMS) {
|
||||
url = hadoopConf.get(KafkaConnectConfigs.HIVE_METASTORE_URIS);
|
||||
} else {
|
||||
url = hiveSyncConfig.jdbcUrl;
|
||||
}
|
||||
|
||||
LOG.info("Syncing target hoodie table with hive table("
|
||||
+ hiveSyncConfig.tableName
|
||||
+ "). Hive URL :"
|
||||
+ url
|
||||
+ ", basePath :" + tableBasePath);
|
||||
LOG.info("Hive Sync Conf => " + hiveSyncConfig);
|
||||
FileSystem fs = FSUtils.getFs(tableBasePath, hadoopConf);
|
||||
HiveConf hiveConf = new HiveConf();
|
||||
hiveConf.addResource(fs.getConf());
|
||||
LOG.info("Hive Conf => " + hiveConf.getAllProperties().toString());
|
||||
new HiveSyncTool(hiveSyncConfig, hiveConf, fs).syncHoodieTable();
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user