[HUDI-3204] fix problem that spark on TimestampKeyGenerator has no re… (#4714)
This commit is contained in:
@@ -19,10 +19,12 @@
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex;
|
||||
import org.apache.hudi.common.config.ConfigClassProperty;
|
||||
@@ -41,12 +43,15 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config;
|
||||
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
@@ -182,6 +187,14 @@ public class HoodieTableConfig extends HoodieConfig {
|
||||
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING = KeyGeneratorOptions.URL_ENCODE_PARTITIONING;
|
||||
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_ENABLE = KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_ENABLE;
|
||||
|
||||
public static final List<String> PERSISTED_CONFIG_LIST = Arrays.asList(
|
||||
Config.DATE_TIME_PARSER_PROP,
|
||||
Config.INPUT_TIME_UNIT, Config.TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP,
|
||||
Config.TIMESTAMP_INPUT_DATE_FORMAT_PROP, Config.TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP,
|
||||
Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP, Config.TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP,
|
||||
Config.TIMESTAMP_TIMEZONE_FORMAT_PROP, Config.DATE_TIME_PARSER_PROP
|
||||
);
|
||||
|
||||
public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
|
||||
|
||||
@@ -54,6 +54,7 @@ import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Objects;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
@@ -643,6 +644,12 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
private Boolean urlEncodePartitioning;
|
||||
private HoodieTimelineTimeZone commitTimeZone;
|
||||
|
||||
/**
|
||||
* Persist the configs that is written at the first time, and should not be changed.
|
||||
* Like KeyGenerator's configs.
|
||||
*/
|
||||
private Properties others = new Properties();
|
||||
|
||||
private PropertyBuilder() {
|
||||
|
||||
}
|
||||
@@ -750,6 +757,23 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
return this;
|
||||
}
|
||||
|
||||
public PropertyBuilder set(String key, Object value) {
|
||||
if (HoodieTableConfig.PERSISTED_CONFIG_LIST.contains(key)) {
|
||||
this.others.put(key, value);
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public PropertyBuilder set(Map<String, Object> props) {
|
||||
for (String key: HoodieTableConfig.PERSISTED_CONFIG_LIST) {
|
||||
Object value = props.get(key);
|
||||
if (value != null) {
|
||||
set(key, value);
|
||||
}
|
||||
}
|
||||
return this;
|
||||
}
|
||||
|
||||
public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) {
|
||||
return setTableType(metaClient.getTableType())
|
||||
.setTableName(metaClient.getTableConfig().getTableName())
|
||||
@@ -759,6 +783,14 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
|
||||
public PropertyBuilder fromProperties(Properties properties) {
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
|
||||
for (String key: HoodieTableConfig.PERSISTED_CONFIG_LIST) {
|
||||
Object value = hoodieConfig.getString(key);
|
||||
if (value != null) {
|
||||
set(key, value);
|
||||
}
|
||||
}
|
||||
|
||||
if (hoodieConfig.contains(HoodieTableConfig.DATABASE_NAME)) {
|
||||
setDatabaseName(hoodieConfig.getString(HoodieTableConfig.DATABASE_NAME));
|
||||
}
|
||||
@@ -828,6 +860,9 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
ValidationUtils.checkArgument(tableName != null, "tableName is null");
|
||||
|
||||
HoodieTableConfig tableConfig = new HoodieTableConfig();
|
||||
|
||||
tableConfig.setAll(others);
|
||||
|
||||
if (databaseName != null) {
|
||||
tableConfig.setValue(HoodieTableConfig.DATABASE_NAME, databaseName);
|
||||
}
|
||||
|
||||
@@ -96,5 +96,28 @@ public class KeyGeneratorOptions extends HoodieConfig {
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String PARTITIONPATH_FIELD_OPT_KEY = PARTITIONPATH_FIELD_NAME.key();
|
||||
|
||||
/**
|
||||
* Supported configs.
|
||||
*/
|
||||
public static class Config {
|
||||
|
||||
// One value from TimestampType above
|
||||
public static final String TIMESTAMP_TYPE_FIELD_PROP = "hoodie.deltastreamer.keygen.timebased.timestamp.type";
|
||||
public static final String INPUT_TIME_UNIT =
|
||||
"hoodie.deltastreamer.keygen.timebased.timestamp.scalar.time.unit";
|
||||
//This prop can now accept list of input date formats.
|
||||
public static final String TIMESTAMP_INPUT_DATE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.input.dateformat";
|
||||
public static final String TIMESTAMP_INPUT_DATE_FORMAT_LIST_DELIMITER_REGEX_PROP = "hoodie.deltastreamer.keygen.timebased.input.dateformat.list.delimiter.regex";
|
||||
public static final String TIMESTAMP_INPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.input.timezone";
|
||||
public static final String TIMESTAMP_OUTPUT_DATE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.output.dateformat";
|
||||
//still keeping this prop for backward compatibility so that functionality for existing users does not break.
|
||||
public static final String TIMESTAMP_TIMEZONE_FORMAT_PROP =
|
||||
"hoodie.deltastreamer.keygen.timebased.timezone";
|
||||
public static final String TIMESTAMP_OUTPUT_TIMEZONE_FORMAT_PROP = "hoodie.deltastreamer.keygen.timebased.output.timezone";
|
||||
public static final String DATE_TIME_PARSER_PROP = "hoodie.deltastreamer.keygen.datetime.parser.class";
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user