1
0

[HUDI-3978] Fix use of partition path field as hive partition field in flink (#5434)

* Fix partition path fields as hive sync partition fields error
This commit is contained in:
Wangyh
2022-04-30 11:58:54 +08:00
committed by GitHub
parent f492c52ee4
commit 33ff4752ba
3 changed files with 79 additions and 2 deletions

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.sink.utils;
import org.apache.flink.annotation.VisibleForTesting;
import org.apache.hudi.aws.sync.AwsGlueCatalogSyncTool;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.configuration.FlinkOptions;
@@ -70,7 +71,8 @@ public class HiveSyncContext {
return new HiveSyncContext(syncConfig, hiveConf, fs);
}
private static HiveSyncConfig buildSyncConfig(Configuration conf) {
@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);
@@ -83,7 +85,7 @@ public class HiveSyncContext {
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.extractPartitionKeys(conf));
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);

View File

@@ -425,4 +425,17 @@ public class FilePathUtils {
}
return conf.getString(FlinkOptions.PARTITION_PATH_FIELD).split(",");
}
/**
* Extracts the hive sync partition fields with given configuration.
*
* @param conf The flink configuration
* @return array of the hive partition fields
*/
public static String[] extractHivePartitionFields(org.apache.flink.configuration.Configuration conf) {
if (FlinkOptions.isDefaultValueDefined(conf, FlinkOptions.HIVE_SYNC_PARTITION_FIELDS)) {
return extractPartitionKeys(conf);
}
return conf.getString(FlinkOptions.HIVE_SYNC_PARTITION_FIELDS).split(",");
}
}