1
0

[HUDI-3730] Improve meta sync class design and hierarchies (#5854)

* [HUDI-3730] Improve meta sync class design and hierarchies (#5754)
* Implements class design proposed in RFC-55

Co-authored-by: jian.feng <fengjian428@gmial.com>
Co-authored-by: jian.feng <jian.feng@shopee.com>
This commit is contained in:
Shiyan Xu
2022-07-03 04:17:25 -05:00
committed by GitHub
parent c00ea84985
commit c0e1587966
86 changed files with 2977 additions and 2877 deletions

View File

@@ -18,22 +18,43 @@
package org.apache.hudi.sink.utils;
import org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool;
import org.apache.hudi.aws.sync.AWSGlueCatalogSyncTool;
import org.apache.hudi.common.config.SerializableConfiguration;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.configuration.HadoopConfigurations;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.hudi.hive.HiveSyncTool;
import org.apache.hudi.hive.ddl.HiveSyncMode;
import org.apache.hudi.table.format.FilePathUtils;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.flink.configuration.Configuration;
import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.hive.conf.HiveConf;
import java.util.Arrays;
import java.util.Properties;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_AUTO_CREATE_DATABASE;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_IGNORE_EXCEPTIONS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SUPPORT_TIMESTAMP_TYPE;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_SYNC_MODE;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_PROPERTIES;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_TABLE_SERDE_PROPERTIES;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_URL;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USER;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_JDBC;
import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_USE_PRE_APACHE_INPUT_FORMAT;
import static org.apache.hudi.hive.HiveSyncConfigHolder.METASTORE_URIS;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_ASSUME_DATE_PARTITION;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_FILE_FORMAT;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_BASE_PATH;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DATABASE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_DECODE_PARTITION;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_EXTRACTOR_CLASS;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_TABLE_NAME;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_USE_FILE_LISTING_FROM_METADATA;
/**
* Hive synchronization context.
@@ -41,63 +62,57 @@ import java.util.Arrays;
* <p>Use this context to create the {@link HiveSyncTool} for synchronization.
*/
public class HiveSyncContext {
private final HiveSyncConfig syncConfig;
private final HiveConf hiveConf;
private final FileSystem fs;
private HiveSyncContext(HiveSyncConfig syncConfig, HiveConf hiveConf, FileSystem fs) {
this.syncConfig = syncConfig;
private final Properties props;
private final HiveConf hiveConf;
private HiveSyncContext(Properties props, HiveConf hiveConf) {
this.props = props;
this.hiveConf = hiveConf;
this.fs = fs;
}
public HiveSyncTool hiveSyncTool() {
HiveSyncMode syncMode = HiveSyncMode.of(syncConfig.syncMode);
HiveSyncMode syncMode = HiveSyncMode.of(props.getProperty(HIVE_SYNC_MODE.key()));
if (syncMode == HiveSyncMode.GLUE) {
return new AwsGlueCatalogSyncTool(this.syncConfig, this.hiveConf, this.fs);
return new AWSGlueCatalogSyncTool(props, hiveConf);
}
return new HiveSyncTool(this.syncConfig, this.hiveConf, this.fs);
return new HiveSyncTool(props, hiveConf);
}
public static HiveSyncContext create(Configuration conf, SerializableConfiguration serConf) {
HiveSyncConfig syncConfig = buildSyncConfig(conf);
Properties props = buildSyncConfig(conf);
org.apache.hadoop.conf.Configuration hadoopConf = HadoopConfigurations.getHadoopConf(conf);
String path = conf.getString(FlinkOptions.PATH);
FileSystem fs = FSUtils.getFs(path, hadoopConf);
HiveConf hiveConf = new HiveConf();
if (!FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_METASTORE_URIS)) {
hadoopConf.set(HiveConf.ConfVars.METASTOREURIS.varname, conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS));
}
hiveConf.addResource(serConf.get());
hiveConf.addResource(hadoopConf);
return new HiveSyncContext(syncConfig, hiveConf, fs);
return new HiveSyncContext(props, hiveConf);
}
@VisibleForTesting
public static HiveSyncConfig buildSyncConfig(Configuration conf) {
HiveSyncConfig hiveSyncConfig = new HiveSyncConfig();
hiveSyncConfig.basePath = conf.getString(FlinkOptions.PATH);
hiveSyncConfig.baseFileFormat = conf.getString(FlinkOptions.HIVE_SYNC_FILE_FORMAT);
hiveSyncConfig.usePreApacheInputFormat = false;
hiveSyncConfig.databaseName = conf.getString(FlinkOptions.HIVE_SYNC_DB);
hiveSyncConfig.tableName = conf.getString(FlinkOptions.HIVE_SYNC_TABLE);
hiveSyncConfig.syncMode = conf.getString(FlinkOptions.HIVE_SYNC_MODE);
hiveSyncConfig.hiveUser = conf.getString(FlinkOptions.HIVE_SYNC_USERNAME);
hiveSyncConfig.hivePass = conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD);
hiveSyncConfig.tableProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES);
hiveSyncConfig.serdeProperties = conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES);
hiveSyncConfig.jdbcUrl = conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL);
hiveSyncConfig.partitionFields = Arrays.asList(FilePathUtils.extractHivePartitionFields(conf));
hiveSyncConfig.partitionValueExtractorClass = conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME);
hiveSyncConfig.useJdbc = conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC);
hiveSyncConfig.useFileListingFromMetadata = conf.getBoolean(FlinkOptions.METADATA_ENABLED);
hiveSyncConfig.ignoreExceptions = conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS);
hiveSyncConfig.supportTimestamp = conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP);
hiveSyncConfig.autoCreateDatabase = conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB);
hiveSyncConfig.decodePartition = conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING);
hiveSyncConfig.skipROSuffix = conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX);
hiveSyncConfig.assumeDatePartitioning = conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION);
hiveSyncConfig.withOperationField = conf.getBoolean(FlinkOptions.CHANGELOG_ENABLED);
return hiveSyncConfig;
public static Properties buildSyncConfig(Configuration conf) {
TypedProperties props = new TypedProperties();
props.setPropertyIfNonNull(META_SYNC_BASE_PATH.key(), conf.getString(FlinkOptions.PATH));
props.setPropertyIfNonNull(META_SYNC_BASE_FILE_FORMAT.key(), conf.getString(FlinkOptions.HIVE_SYNC_FILE_FORMAT));
props.setPropertyIfNonNull(HIVE_USE_PRE_APACHE_INPUT_FORMAT.key(), "false");
props.setPropertyIfNonNull(META_SYNC_DATABASE_NAME.key(), conf.getString(FlinkOptions.HIVE_SYNC_DB));
props.setPropertyIfNonNull(META_SYNC_TABLE_NAME.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE));
props.setPropertyIfNonNull(HIVE_SYNC_MODE.key(), conf.getString(FlinkOptions.HIVE_SYNC_MODE));
props.setPropertyIfNonNull(HIVE_USER.key(), conf.getString(FlinkOptions.HIVE_SYNC_USERNAME));
props.setPropertyIfNonNull(HIVE_PASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PASSWORD));
props.setPropertyIfNonNull(HIVE_URL.key(), conf.getString(FlinkOptions.HIVE_SYNC_JDBC_URL));
props.setPropertyIfNonNull(METASTORE_URIS.key(), conf.getString(FlinkOptions.HIVE_SYNC_METASTORE_URIS));
props.setPropertyIfNonNull(HIVE_TABLE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_PROPERTIES));
props.setPropertyIfNonNull(HIVE_TABLE_SERDE_PROPERTIES.key(), conf.getString(FlinkOptions.HIVE_SYNC_TABLE_SERDE_PROPERTIES));
props.setPropertyIfNonNull(META_SYNC_PARTITION_FIELDS.key(), String.join(",", FilePathUtils.extractHivePartitionFields(conf)));
props.setPropertyIfNonNull(META_SYNC_PARTITION_EXTRACTOR_CLASS.key(), conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME));
props.setPropertyIfNonNull(HIVE_USE_JDBC.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_USE_JDBC)));
props.setPropertyIfNonNull(META_SYNC_USE_FILE_LISTING_FROM_METADATA.key(), String.valueOf(conf.getBoolean(FlinkOptions.METADATA_ENABLED)));
props.setPropertyIfNonNull(HIVE_IGNORE_EXCEPTIONS.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_IGNORE_EXCEPTIONS)));
props.setPropertyIfNonNull(HIVE_SUPPORT_TIMESTAMP_TYPE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SUPPORT_TIMESTAMP)));
props.setPropertyIfNonNull(HIVE_AUTO_CREATE_DATABASE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_AUTO_CREATE_DB)));
props.setPropertyIfNonNull(META_SYNC_DECODE_PARTITION.key(), String.valueOf(conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING)));
props.setPropertyIfNonNull(HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_SKIP_RO_SUFFIX)));
props.setPropertyIfNonNull(META_SYNC_ASSUME_DATE_PARTITION.key(), String.valueOf(conf.getBoolean(FlinkOptions.HIVE_SYNC_ASSUME_DATE_PARTITION)));
return props;
}
}

View File

@@ -19,14 +19,15 @@
package org.apache.hudi.sink.utils;
import org.apache.hudi.configuration.FlinkOptions;
import org.apache.hudi.hive.HiveSyncConfig;
import org.apache.flink.configuration.Configuration;
import org.junit.jupiter.api.Test;
import java.lang.reflect.Method;
import java.util.Properties;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.apache.hudi.sync.common.HoodieSyncConfig.META_SYNC_PARTITION_FIELDS;
import static org.junit.jupiter.api.Assertions.assertEquals;
/**
* Test cases for {@link HiveSyncContext}.
@@ -51,11 +52,11 @@ public class TestHiveSyncContext {
Method buildSyncConfigMethod = threadClazz.getDeclaredMethod("buildSyncConfig", Configuration.class);
buildSyncConfigMethod.setAccessible(true);
HiveSyncConfig hiveSyncConfig1 = HiveSyncContext.buildSyncConfig(configuration1);
HiveSyncConfig hiveSyncConfig2 = HiveSyncContext.buildSyncConfig(configuration2);
Properties props1 = HiveSyncContext.buildSyncConfig(configuration1);
Properties props2 = HiveSyncContext.buildSyncConfig(configuration2);
assertTrue(hiveSyncConfig1.partitionFields.get(0).equals(hiveSyncPartitionField));
assertTrue(hiveSyncConfig2.partitionFields.get(0).equals(partitionPathField));
assertEquals(hiveSyncPartitionField, props1.getProperty(META_SYNC_PARTITION_FIELDS.key()));
assertEquals(partitionPathField, props2.getProperty(META_SYNC_PARTITION_FIELDS.key()));
}
}