Skip to content
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -172,6 +172,11 @@ public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty) {
.orElseGet(() -> Boolean.parseBoolean(configProperty.defaultValue().toString()));
}

public <T> boolean getBooleanOrDefault(ConfigProperty<T> configProperty, boolean defaultVal) {
Option<Object> rawValue = getRawValue(configProperty);
return rawValue.map(v -> Boolean.parseBoolean(v.toString())).orElse(defaultVal);
}

public <T> Long getLong(ConfigProperty<T> configProperty) {
Option<Object> rawValue = getRawValue(configProperty);
return rawValue.map(v -> Long.parseLong(v.toString())).orElse(null);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -426,12 +426,8 @@ object DataSourceWriteOptions {
@Deprecated
val METASTORE_URIS: ConfigProperty[String] = HiveSyncConfigHolder.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
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

these infer functions shouldn't be here for users to import in the first place

@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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.config.TypedProperties;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.table.HoodieTableConfig;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.StringUtils;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
Expand All @@ -34,8 +35,8 @@

import java.util.List;
import java.util.Properties;
import java.util.function.Function;

import static org.apache.hudi.common.config.HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS;
import static org.apache.hudi.common.table.HoodieTableConfig.DATABASE_NAME;
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_TABLE_NAME_KEY;
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_WRITE_TABLE_NAME_KEY;
Expand Down Expand Up @@ -72,57 +73,51 @@ public class HoodieSyncConfig extends HoodieConfig {
public static final ConfigProperty<String> META_SYNC_BASE_FILE_FORMAT = ConfigProperty
.key("hoodie.datasource.hive_sync.base_file_format")
.defaultValue("PARQUET")
.withInferFunction(cfg -> Option.ofNullable(cfg.getString(HoodieTableConfig.BASE_FILE_FORMAT)))
.withDocumentation("Base file format for the sync.");

// If partition fields are not explicitly provided, obtain from the KeyGeneration Configs
public static final Function<HoodieConfig, Option<String>> 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<String> META_SYNC_PARTITION_FIELDS = ConfigProperty
.key("hoodie.datasource.hive_sync.partition_fields")
.defaultValue("")
.withInferFunction(PARTITION_FIELDS_INFERENCE_FUNCTION)
.withInferFunction(cfg -> Option.ofNullable(cfg.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)))
.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<HoodieConfig, Option<String>> 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<String> 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)
.withInferFunction(cfg -> {
if (cfg.contains(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME)) {
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");
}
} else {
return Option.of("org.apache.hudi.hive.NonPartitionedExtractor");
}
})
.withDocumentation("Class which implements PartitionValueExtractor to extract the partition values, "
+ "default 'SlashEncodedDayPartitionValueExtractor'.");

public static final ConfigProperty<String> META_SYNC_ASSUME_DATE_PARTITION = ConfigProperty
.key("hoodie.datasource.hive_sync.assume_date_partitioning")
.defaultValue("false")
.withDocumentation("Assume partitioning is yyyy/mm/dd");
.defaultValue(HoodieMetadataConfig.ASSUME_DATE_PARTITIONING.defaultValue())
Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

also "false"

.withInferFunction(cfg -> Option.ofNullable(cfg.getString(HoodieMetadataConfig.ASSUME_DATE_PARTITIONING)))
.withDocumentation("Assume partitioning is yyyy/MM/dd");

public static final ConfigProperty<Boolean> META_SYNC_DECODE_PARTITION = ConfigProperty
.key("hoodie.meta.sync.decode_partition")
.defaultValue(false) // TODO infer from url encode option
.withDocumentation("");
.defaultValue(false)
.withInferFunction(cfg -> Option.ofNullable(cfg.getBoolean(HoodieTableConfig.URL_ENCODE_PARTITIONING)))
.withDocumentation("If true, meta sync will url-decode the partition path, as it is deemed as url-encoded. Default to false.");

public static final ConfigProperty<Boolean> META_SYNC_USE_FILE_LISTING_FROM_METADATA = ConfigProperty
.key("hoodie.meta.sync.metadata_file_listing")
.defaultValue(HoodieMetadataConfig.DEFAULT_METADATA_ENABLE_FOR_READERS)
.defaultValue(DEFAULT_METADATA_ENABLE_FOR_READERS)
.withInferFunction(cfg -> Option.of(cfg.getBooleanOrDefault(HoodieMetadataConfig.ENABLE, DEFAULT_METADATA_ENABLE_FOR_READERS)))
.withDocumentation("Enable the internal metadata table for file listing for syncing with metastores");

public static final ConfigProperty<String> META_SYNC_CONDITIONAL_SYNC = ConfigProperty
Expand Down