[HUDI-89] Add configOption & refactor all configs based on that (#2833)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -0,0 +1,154 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.config;
|
||||
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.function.Function;
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* ConfigProperty describes a configuration property. It contains the configuration
|
||||
* key, deprecated older versions of the key, and an optional default value for the configuration,
|
||||
* configuration descriptions and also the an infer mechanism to infer the configuration value
|
||||
* based on other configurations.
|
||||
*
|
||||
* @param <T> The type of the default value.
|
||||
*/
|
||||
public class ConfigProperty<T> implements Serializable {
|
||||
|
||||
private final String key;
|
||||
|
||||
private final T defaultValue;
|
||||
|
||||
private final String doc;
|
||||
|
||||
private final Option<String> sinceVersion;
|
||||
|
||||
private final Option<String> deprecatedVersion;
|
||||
|
||||
private final String[] alternatives;
|
||||
|
||||
// provide the ability to infer config value based on other configs
|
||||
private final Option<Function<HoodieConfig, Option<T>>> inferFunction;
|
||||
|
||||
ConfigProperty(String key, T defaultValue, String doc, Option<String> sinceVersion,
|
||||
Option<String> deprecatedVersion, Option<Function<HoodieConfig, Option<T>>> inferFunc, String... alternatives) {
|
||||
this.key = Objects.requireNonNull(key);
|
||||
this.defaultValue = defaultValue;
|
||||
this.doc = doc;
|
||||
this.sinceVersion = sinceVersion;
|
||||
this.deprecatedVersion = deprecatedVersion;
|
||||
this.inferFunction = inferFunc;
|
||||
this.alternatives = alternatives;
|
||||
}
|
||||
|
||||
public String key() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public T defaultValue() {
|
||||
if (defaultValue == null) {
|
||||
throw new HoodieException("There's no default value for this config");
|
||||
}
|
||||
return defaultValue;
|
||||
}
|
||||
|
||||
public boolean hasDefaultValue() {
|
||||
return defaultValue != null;
|
||||
}
|
||||
|
||||
Option<Function<HoodieConfig, Option<T>>> getInferFunc() {
|
||||
return inferFunction;
|
||||
}
|
||||
|
||||
public String[] getAlternatives() {
|
||||
return alternatives;
|
||||
}
|
||||
|
||||
public ConfigProperty<T> withDocumentation(String doc) {
|
||||
Objects.requireNonNull(doc);
|
||||
return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives);
|
||||
}
|
||||
|
||||
public ConfigProperty<T> withAlternatives(String... alternatives) {
|
||||
Objects.requireNonNull(alternatives);
|
||||
return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, inferFunction, alternatives);
|
||||
}
|
||||
|
||||
public ConfigProperty<T> sinceVersion(String sinceVersion) {
|
||||
Objects.requireNonNull(sinceVersion);
|
||||
return new ConfigProperty<>(key, defaultValue, doc, Option.of(sinceVersion), deprecatedVersion, inferFunction, alternatives);
|
||||
}
|
||||
|
||||
public ConfigProperty<T> deprecatedAfter(String deprecatedVersion) {
|
||||
Objects.requireNonNull(deprecatedVersion);
|
||||
return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, Option.of(deprecatedVersion), inferFunction, alternatives);
|
||||
}
|
||||
|
||||
public ConfigProperty<T> withInferFunction(Function<HoodieConfig, Option<T>> inferFunction) {
|
||||
Objects.requireNonNull(inferFunction);
|
||||
return new ConfigProperty<>(key, defaultValue, doc, sinceVersion, deprecatedVersion, Option.of(inferFunction), alternatives);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a OptionBuilder with key.
|
||||
*
|
||||
* @param key The key of the option
|
||||
* @return Return a OptionBuilder.
|
||||
*/
|
||||
public static PropertyBuilder key(String key) {
|
||||
Objects.requireNonNull(key);
|
||||
return new PropertyBuilder(key);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return String.format(
|
||||
"Key: '%s' , default: %s description: %s since version: %s deprecated after: %s)",
|
||||
key, defaultValue, doc, sinceVersion.isPresent() ? sinceVersion.get() : "version is not defined",
|
||||
deprecatedVersion.isPresent() ? deprecatedVersion.get() : "version is not defined");
|
||||
}
|
||||
|
||||
/**
|
||||
* The PropertyBuilder is used to build the ConfigProperty.
|
||||
*/
|
||||
public static final class PropertyBuilder {
|
||||
|
||||
private final String key;
|
||||
|
||||
PropertyBuilder(String key) {
|
||||
this.key = key;
|
||||
}
|
||||
|
||||
public <T> ConfigProperty<T> defaultValue(T value) {
|
||||
Objects.requireNonNull(value);
|
||||
ConfigProperty<T> configProperty = new ConfigProperty<>(key, value, "", Option.empty(), Option.empty(), Option.empty());
|
||||
return configProperty;
|
||||
}
|
||||
|
||||
public ConfigProperty<String> noDefaultValue() {
|
||||
ConfigProperty<String> configProperty = new ConfigProperty<>(key, null, "", Option.empty(),
|
||||
Option.empty(), Option.empty());
|
||||
return configProperty;
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,51 +0,0 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.config;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Default Way to load Hoodie config through a {@link java.util.Properties}.
|
||||
*/
|
||||
public class DefaultHoodieConfig implements Serializable {
|
||||
|
||||
protected final Properties props;
|
||||
|
||||
public DefaultHoodieConfig(Properties props) {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
public static void setDefaultOnCondition(Properties props, boolean condition, String propName, String defaultValue) {
|
||||
if (condition) {
|
||||
props.setProperty(propName, defaultValue);
|
||||
}
|
||||
}
|
||||
|
||||
public static void setDefaultOnCondition(Properties props, boolean condition, DefaultHoodieConfig config) {
|
||||
if (condition) {
|
||||
props.putAll(config.getProps());
|
||||
}
|
||||
}
|
||||
|
||||
public Properties getProps() {
|
||||
return props;
|
||||
}
|
||||
|
||||
}
|
||||
@@ -0,0 +1,177 @@
|
||||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hudi.common.config;
|
||||
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ReflectionUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.lang.reflect.Modifier;
|
||||
import java.util.Arrays;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* This class deals with {@link ConfigProperty} and provides get/set functionalities.
|
||||
*/
|
||||
public class HoodieConfig implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieConfig.class);
|
||||
|
||||
public static HoodieConfig create(FSDataInputStream inputStream) throws IOException {
|
||||
HoodieConfig config = new HoodieConfig();
|
||||
config.props.load(inputStream);
|
||||
return config;
|
||||
}
|
||||
|
||||
protected Properties props;
|
||||
|
||||
public HoodieConfig() {
|
||||
this.props = new Properties();
|
||||
}
|
||||
|
||||
public HoodieConfig(Properties props) {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
public <T> void setValue(ConfigProperty<T> cfg, String val) {
|
||||
props.setProperty(cfg.key(), val);
|
||||
}
|
||||
|
||||
public <T> void setDefaultValue(ConfigProperty<T> configProperty) {
|
||||
if (!contains(configProperty)) {
|
||||
Option<T> inferValue = Option.empty();
|
||||
if (configProperty.getInferFunc().isPresent()) {
|
||||
inferValue = configProperty.getInferFunc().get().apply(this);
|
||||
}
|
||||
props.setProperty(configProperty.key(), inferValue.isPresent() ? inferValue.get().toString() : configProperty.defaultValue().toString());
|
||||
}
|
||||
}
|
||||
|
||||
public <T> void setDefaultValue(ConfigProperty<T> configProperty, T defaultVal) {
|
||||
if (!contains(configProperty)) {
|
||||
props.setProperty(configProperty.key(), defaultVal.toString());
|
||||
}
|
||||
}
|
||||
|
||||
public <T> boolean contains(ConfigProperty<T> configProperty) {
|
||||
if (props.containsKey(configProperty.key())) {
|
||||
return true;
|
||||
}
|
||||
return Arrays.stream(configProperty.getAlternatives()).anyMatch(props::containsKey);
|
||||
}
|
||||
|
||||
private <T> Option<Object> getRawValue(ConfigProperty<T> configProperty) {
|
||||
if (props.containsKey(configProperty.key())) {
|
||||
return Option.ofNullable(props.get(configProperty.key()));
|
||||
}
|
||||
for (String alternative : configProperty.getAlternatives()) {
|
||||
if (props.containsKey(alternative)) {
|
||||
LOG.warn(String.format("The configuration key '%s' has been deprecated "
|
||||
+ "and may be removed in the future. Please use the new key '%s' instead.",
|
||||
alternative, configProperty.key()));
|
||||
return Option.ofNullable(props.get(alternative));
|
||||
}
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
protected void setDefaults(String configClassName) {
|
||||
Class<?> configClass = ReflectionUtils.getClass(configClassName);
|
||||
Arrays.stream(configClass.getDeclaredFields())
|
||||
.filter(f -> Modifier.isStatic(f.getModifiers()))
|
||||
.filter(f -> f.getType().isAssignableFrom(ConfigProperty.class))
|
||||
.forEach(f -> {
|
||||
try {
|
||||
ConfigProperty<?> cfgProp = (ConfigProperty<?>) f.get("null");
|
||||
if (cfgProp.hasDefaultValue()) {
|
||||
setDefaultValue(cfgProp);
|
||||
}
|
||||
} catch (IllegalAccessException e) {
|
||||
e.printStackTrace();
|
||||
}
|
||||
});
|
||||
}
|
||||
|
||||
public <T> String getString(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(Object::toString).orElse(null);
|
||||
}
|
||||
|
||||
public String getString(String key) {
|
||||
return props.getProperty(key);
|
||||
}
|
||||
|
||||
public <T> Integer getInt(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(v -> Integer.parseInt(v.toString())).orElse(null);
|
||||
}
|
||||
|
||||
public <T> Boolean getBoolean(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(v -> Boolean.parseBoolean(v.toString())).orElse(null);
|
||||
}
|
||||
|
||||
public <T> Long getLong(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(v -> Long.parseLong(v.toString())).orElse(null);
|
||||
}
|
||||
|
||||
public <T> Float getFloat(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(v -> Float.parseFloat(v.toString())).orElse(null);
|
||||
}
|
||||
|
||||
public <T> Double getDouble(ConfigProperty<T> configProperty) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(v -> Double.parseDouble(v.toString())).orElse(null);
|
||||
}
|
||||
|
||||
public <T> String getStringOrDefault(ConfigProperty<T> configProperty) {
|
||||
return getStringOrDefault(configProperty, configProperty.defaultValue().toString());
|
||||
}
|
||||
|
||||
public <T> String getStringOrDefault(ConfigProperty<T> configProperty, String defaultVal) {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
return rawValue.map(Object::toString).orElse(defaultVal);
|
||||
}
|
||||
|
||||
public Properties getProps() {
|
||||
return props;
|
||||
}
|
||||
|
||||
public void setDefaultOnCondition(boolean condition, HoodieConfig config) {
|
||||
if (condition) {
|
||||
props.putAll(config.getProps());
|
||||
}
|
||||
}
|
||||
|
||||
public <T> String getStringOrThrow(ConfigProperty<T> configProperty, String errorMessage) throws HoodieException {
|
||||
Option<Object> rawValue = getRawValue(configProperty);
|
||||
if (rawValue.isPresent()) {
|
||||
return rawValue.get().toString();
|
||||
} else {
|
||||
throw new HoodieException(errorMessage);
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -28,57 +28,96 @@ import java.util.Properties;
|
||||
* Configurations used by the HUDI Metadata Table.
|
||||
*/
|
||||
@Immutable
|
||||
public final class HoodieMetadataConfig extends DefaultHoodieConfig {
|
||||
public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
|
||||
public static final String METADATA_PREFIX = "hoodie.metadata";
|
||||
|
||||
// Enable the internal Metadata Table which saves file listings
|
||||
public static final String METADATA_ENABLE_PROP = METADATA_PREFIX + ".enable";
|
||||
public static final boolean DEFAULT_METADATA_ENABLE = false;
|
||||
public static final ConfigProperty<Boolean> METADATA_ENABLE_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable the internal Metadata Table which stores table level file listings");
|
||||
|
||||
// Validate contents of Metadata Table on each access against the actual filesystem
|
||||
public static final String METADATA_VALIDATE_PROP = METADATA_PREFIX + ".validate";
|
||||
public static final boolean DEFAULT_METADATA_VALIDATE = false;
|
||||
public static final ConfigProperty<Boolean> METADATA_VALIDATE_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".validate")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Validate contents of Metadata Table on each access against the actual listings from DFS");
|
||||
|
||||
public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
|
||||
|
||||
// Enable metrics for internal Metadata Table
|
||||
public static final String METADATA_METRICS_ENABLE_PROP = METADATA_PREFIX + ".metrics.enable";
|
||||
public static final boolean DEFAULT_METADATA_METRICS_ENABLE = false;
|
||||
public static final ConfigProperty<Boolean> METADATA_METRICS_ENABLE_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".metrics.enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// Parallelism for inserts
|
||||
public static final String METADATA_INSERT_PARALLELISM_PROP = METADATA_PREFIX + ".insert.parallelism";
|
||||
public static final int DEFAULT_METADATA_INSERT_PARALLELISM = 1;
|
||||
public static final ConfigProperty<Integer> METADATA_INSERT_PARALLELISM_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".insert.parallelism")
|
||||
.defaultValue(1)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Parallelism to use when writing to the metadata table");
|
||||
|
||||
// Async clean
|
||||
public static final String METADATA_ASYNC_CLEAN_PROP = METADATA_PREFIX + ".clean.async";
|
||||
public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = false;
|
||||
public static final ConfigProperty<Boolean> METADATA_ASYNC_CLEAN_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".clean.async")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable asynchronous cleaning for metadata table");
|
||||
|
||||
// Maximum delta commits before compaction occurs
|
||||
public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = METADATA_PREFIX + ".compact.max.delta.commits";
|
||||
public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = 24;
|
||||
public static final ConfigProperty<Integer> METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".compact.max.delta.commits")
|
||||
.defaultValue(24)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls how often the metadata table is compacted.");
|
||||
|
||||
// Archival settings
|
||||
public static final String MIN_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.min.commits";
|
||||
public static final int DEFAULT_MIN_COMMITS_TO_KEEP = 20;
|
||||
public static final String MAX_COMMITS_TO_KEEP_PROP = METADATA_PREFIX + ".keep.max.commits";
|
||||
public static final int DEFAULT_MAX_COMMITS_TO_KEEP = 30;
|
||||
public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".keep.min.commits")
|
||||
.defaultValue(20)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline");
|
||||
|
||||
public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".keep.max.commits")
|
||||
.defaultValue(30)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline");
|
||||
|
||||
// Cleaner commits retained
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = METADATA_PREFIX + ".cleaner.commits.retained";
|
||||
public static final int DEFAULT_CLEANER_COMMITS_RETAINED = 3;
|
||||
public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".cleaner.commits.retained")
|
||||
.defaultValue(3)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// Regex to filter out matching directories during bootstrap
|
||||
public static final String DIRECTORY_FILTER_REGEX = METADATA_PREFIX + ".dir.filter.regex";
|
||||
public static final String DEFAULT_DIRECTORY_FILTER_REGEX = "";
|
||||
public static final ConfigProperty<String> DIRECTORY_FILTER_REGEX = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".dir.filter.regex")
|
||||
.defaultValue("")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = "hoodie.assume.date.partitioning";
|
||||
public static final String DEFAULT_ASSUME_DATE_PARTITIONING = "false";
|
||||
public static final ConfigProperty<String> HOODIE_ASSUME_DATE_PARTITIONING_PROP = ConfigProperty
|
||||
.key("hoodie.assume.date.partitioning")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. "
|
||||
+ "This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually");
|
||||
|
||||
public static final String FILE_LISTING_PARALLELISM_PROP = "hoodie.file.listing.parallelism";
|
||||
public static final int DEFAULT_FILE_LISTING_PARALLELISM = 1500;
|
||||
public static final ConfigProperty<Integer> FILE_LISTING_PARALLELISM_PROP = ConfigProperty
|
||||
.key("hoodie.file.listing.parallelism")
|
||||
.defaultValue(1500)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
|
||||
private HoodieMetadataConfig(Properties props) {
|
||||
super(props);
|
||||
private HoodieMetadataConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieMetadataConfig.Builder newBuilder() {
|
||||
@@ -86,128 +125,104 @@ public final class HoodieMetadataConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public int getFileListingParallelism() {
|
||||
return Math.max(Integer.parseInt(props.getProperty(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP)), 1);
|
||||
return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP), 1);
|
||||
}
|
||||
|
||||
public Boolean shouldAssumeDatePartitioning() {
|
||||
return Boolean.parseBoolean(props.getProperty(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP));
|
||||
return getBoolean(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP);
|
||||
}
|
||||
|
||||
public boolean useFileListingMetadata() {
|
||||
return Boolean.parseBoolean(props.getProperty(METADATA_ENABLE_PROP));
|
||||
return getBoolean(METADATA_ENABLE_PROP);
|
||||
}
|
||||
|
||||
public boolean validateFileListingMetadata() {
|
||||
return Boolean.parseBoolean(props.getProperty(METADATA_VALIDATE_PROP));
|
||||
return getBoolean(METADATA_VALIDATE_PROP);
|
||||
}
|
||||
|
||||
public boolean enableMetrics() {
|
||||
return Boolean.parseBoolean(props.getProperty(METADATA_METRICS_ENABLE_PROP));
|
||||
return getBoolean(METADATA_METRICS_ENABLE_PROP);
|
||||
}
|
||||
|
||||
public String getDirectoryFilterRegex() {
|
||||
return props.getProperty(DIRECTORY_FILTER_REGEX);
|
||||
return getString(DIRECTORY_FILTER_REGEX);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieMetadataConfig metadataConfig = new HoodieMetadataConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.metadataConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.metadataConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder enable(boolean enable) {
|
||||
props.setProperty(METADATA_ENABLE_PROP, String.valueOf(enable));
|
||||
metadataConfig.setValue(METADATA_ENABLE_PROP, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder enableMetrics(boolean enableMetrics) {
|
||||
props.setProperty(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
|
||||
metadataConfig.setValue(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder validate(boolean validate) {
|
||||
props.setProperty(METADATA_VALIDATE_PROP, String.valueOf(validate));
|
||||
metadataConfig.setValue(METADATA_VALIDATE_PROP, String.valueOf(validate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInsertParallelism(int parallelism) {
|
||||
props.setProperty(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
metadataConfig.setValue(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncClean(boolean asyncClean) {
|
||||
props.setProperty(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
metadataConfig.setValue(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
props.setProperty(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
metadataConfig.setValue(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
||||
props.setProperty(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
props.setProperty(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
metadataConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
metadataConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainCommits(int commitsRetained) {
|
||||
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
metadataConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFileListingParallelism(int parallelism) {
|
||||
props.setProperty(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
metadataConfig.setValue(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
||||
props.setProperty(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
|
||||
metadataConfig.setValue(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDirectoryFilterRegex(String regex) {
|
||||
props.setProperty(DIRECTORY_FILTER_REGEX, regex);
|
||||
metadataConfig.setValue(DIRECTORY_FILTER_REGEX, regex);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetadataConfig build() {
|
||||
HoodieMetadataConfig config = new HoodieMetadataConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_ENABLE_PROP), METADATA_ENABLE_PROP,
|
||||
String.valueOf(DEFAULT_METADATA_ENABLE));
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_METRICS_ENABLE_PROP), METADATA_METRICS_ENABLE_PROP,
|
||||
String.valueOf(DEFAULT_METADATA_METRICS_ENABLE));
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_VALIDATE_PROP), METADATA_VALIDATE_PROP,
|
||||
String.valueOf(DEFAULT_METADATA_VALIDATE));
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_INSERT_PARALLELISM_PROP), METADATA_INSERT_PARALLELISM_PROP,
|
||||
String.valueOf(DEFAULT_METADATA_INSERT_PARALLELISM));
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_ASYNC_CLEAN_PROP), METADATA_ASYNC_CLEAN_PROP,
|
||||
String.valueOf(DEFAULT_METADATA_ASYNC_CLEAN));
|
||||
setDefaultOnCondition(props, !props.containsKey(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||
METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS));
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
|
||||
String.valueOf(DEFAULT_CLEANER_COMMITS_RETAINED));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
|
||||
String.valueOf(DEFAULT_MAX_COMMITS_TO_KEEP));
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
|
||||
String.valueOf(DEFAULT_MIN_COMMITS_TO_KEEP));
|
||||
setDefaultOnCondition(props, !props.containsKey(FILE_LISTING_PARALLELISM_PROP), FILE_LISTING_PARALLELISM_PROP,
|
||||
String.valueOf(DEFAULT_FILE_LISTING_PARALLELISM));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_ASSUME_DATE_PARTITIONING_PROP),
|
||||
HOODIE_ASSUME_DATE_PARTITIONING_PROP, DEFAULT_ASSUME_DATE_PARTITIONING);
|
||||
setDefaultOnCondition(props, !props.containsKey(DIRECTORY_FILTER_REGEX), DIRECTORY_FILTER_REGEX,
|
||||
DEFAULT_DIRECTORY_FILTER_REGEX);
|
||||
return config;
|
||||
metadataConfig.setDefaults(HoodieMetadataConfig.class.getName());
|
||||
return metadataConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -28,36 +28,51 @@ import java.util.Properties;
|
||||
public class LockConfiguration implements Serializable {
|
||||
|
||||
public static final String LOCK_PREFIX = "hoodie.write.lock.";
|
||||
public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "wait_time_ms_between_retry";
|
||||
|
||||
public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "wait_time_ms_between_retry";
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
|
||||
public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "max_wait_time_ms_between_retry";
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
|
||||
public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_PREFIX + "client.wait_time_ms_between_retry";
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(10000L);
|
||||
public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_PREFIX + "num_retries";
|
||||
|
||||
public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "max_wait_time_ms_between_retry";
|
||||
|
||||
public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY = LOCK_PREFIX + "client.wait_time_ms_between_retry";
|
||||
|
||||
public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "num_retries";
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_NUM_RETRIES = String.valueOf(3);
|
||||
public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_PREFIX + "client.num_retries";
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES = String.valueOf(0);
|
||||
public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_PREFIX + "wait_time_ms";
|
||||
public static final int DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS = 60 * 1000;
|
||||
|
||||
public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY = LOCK_PREFIX + "client.num_retries";
|
||||
|
||||
public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY = LOCK_PREFIX + "wait_time_ms";
|
||||
|
||||
// configs for file system based locks. NOTE: This only works for DFS with atomic create/delete operation
|
||||
public static final String FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "filesystem.";
|
||||
public static final String FILESYSTEM_LOCK_PATH_PROP = FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX + "path";
|
||||
|
||||
public static final String FILESYSTEM_LOCK_PATH_PROP_KEY = FILESYSTEM_BASED_LOCK_PROPERTY_PREFIX + "path";
|
||||
|
||||
// configs for metastore based locks
|
||||
public static final String HIVE_METASTORE_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "hivemetastore.";
|
||||
public static final String HIVE_DATABASE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "database";
|
||||
public static final String HIVE_TABLE_NAME_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "table";
|
||||
public static final String HIVE_METASTORE_URI_PROP = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "uris";
|
||||
|
||||
public static final String HIVE_DATABASE_NAME_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "database";
|
||||
|
||||
public static final String HIVE_TABLE_NAME_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "table";
|
||||
|
||||
public static final String HIVE_METASTORE_URI_PROP_KEY = HIVE_METASTORE_LOCK_PROPERTY_PREFIX + "uris";
|
||||
|
||||
// Zookeeper configs for zk based locks
|
||||
public static final String ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX = LOCK_PREFIX + "zookeeper.";
|
||||
public static final String ZK_BASE_PATH_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "base_path";
|
||||
public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "session_timeout_ms";
|
||||
|
||||
public static final String ZK_BASE_PATH_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "base_path";
|
||||
|
||||
public static final String ZK_SESSION_TIMEOUT_MS_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "session_timeout_ms";
|
||||
public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
|
||||
public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "connection_timeout_ms";
|
||||
|
||||
public static final String ZK_CONNECTION_TIMEOUT_MS_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "connection_timeout_ms";
|
||||
public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
|
||||
public static final String ZK_CONNECT_URL_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "url";
|
||||
public static final String ZK_PORT_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "port";
|
||||
public static final String ZK_LOCK_KEY_PROP = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
|
||||
|
||||
public static final String ZK_CONNECT_URL_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "url";
|
||||
|
||||
public static final String ZK_PORT_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "port";
|
||||
|
||||
public static final String ZK_LOCK_KEY_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
|
||||
|
||||
private final TypedProperties props;
|
||||
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.common.fs;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
@@ -28,34 +29,53 @@ import java.util.Properties;
|
||||
/**
|
||||
* The consistency guard relevant config options.
|
||||
*/
|
||||
public class ConsistencyGuardConfig extends DefaultHoodieConfig {
|
||||
|
||||
private static final String CONSISTENCY_CHECK_ENABLED_PROP = "hoodie.consistency.check.enabled";
|
||||
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = "false";
|
||||
public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
|
||||
// time between successive attempts to ensure written data's metadata is consistent on storage
|
||||
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP =
|
||||
"hoodie.consistency.check.initial_interval_ms";
|
||||
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = 400L;
|
||||
@Deprecated
|
||||
public static final ConfigProperty<String> CONSISTENCY_CHECK_ENABLED_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.enabled")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("Enabled to handle S3 eventual consistency issue. This property is no longer required "
|
||||
+ "since S3 is now strongly consistent. Will be removed in the future releases.");
|
||||
|
||||
public static final ConfigProperty<Long> INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.initial_interval_ms")
|
||||
.defaultValue(400L)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// max interval time
|
||||
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = "hoodie.consistency.check.max_interval_ms";
|
||||
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = 20000L;
|
||||
public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.max_interval_ms")
|
||||
.defaultValue(20000L)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// maximum number of checks, for consistency of written data. Will wait upto 140 Secs
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = "hoodie.consistency.check.max_checks";
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = 6;
|
||||
public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.max_checks")
|
||||
.defaultValue(6)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// sleep time for OptimisticConsistencyGuard
|
||||
private static final String OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = "hoodie.optimistic.consistency.guard.sleep_time_ms";
|
||||
private static long DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = 500L;
|
||||
public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = ConfigProperty
|
||||
.key("hoodie.optimistic.consistency.guard.sleep_time_ms")
|
||||
.defaultValue(500L)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// config to enable OptimisticConsistencyGuard in finalizeWrite instead of FailSafeConsistencyGuard
|
||||
private static final String ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = "_hoodie.optimistic.consistency.guard.enable";
|
||||
private static boolean DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = true;
|
||||
public static final ConfigProperty<Boolean> ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP = ConfigProperty
|
||||
.key("_hoodie.optimistic.consistency.guard.enable")
|
||||
.defaultValue(true)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public ConsistencyGuardConfig(Properties props) {
|
||||
super(props);
|
||||
private ConsistencyGuardConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static ConsistencyGuardConfig.Builder newBuilder() {
|
||||
@@ -63,27 +83,27 @@ public class ConsistencyGuardConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public boolean isConsistencyCheckEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(CONSISTENCY_CHECK_ENABLED_PROP));
|
||||
return getBoolean(CONSISTENCY_CHECK_ENABLED_PROP);
|
||||
}
|
||||
|
||||
public int getMaxConsistencyChecks() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECKS_PROP));
|
||||
return getInt(MAX_CONSISTENCY_CHECKS_PROP);
|
||||
}
|
||||
|
||||
public int getInitialConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
return getInt(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
|
||||
}
|
||||
|
||||
public int getMaxConsistencyCheckIntervalMs() {
|
||||
return Integer.parseInt(props.getProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP));
|
||||
return getInt(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
|
||||
}
|
||||
|
||||
public long getOptimisticConsistencyGuardSleepTimeMs() {
|
||||
return Long.parseLong(props.getProperty(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP));
|
||||
return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP);
|
||||
}
|
||||
|
||||
public boolean shouldEnableOptimisticConsistencyGuard() {
|
||||
return Boolean.parseBoolean(props.getProperty(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD));
|
||||
return getBoolean(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -91,65 +111,53 @@ public class ConsistencyGuardConfig extends DefaultHoodieConfig {
|
||||
*/
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final ConsistencyGuardConfig consistencyGuardConfig = new ConsistencyGuardConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
props.load(reader);
|
||||
consistencyGuardConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.consistencyGuardConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withConsistencyCheckEnabled(boolean enabled) {
|
||||
props.setProperty(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
|
||||
consistencyGuardConfig.setValue(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
|
||||
props.setProperty(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
|
||||
consistencyGuardConfig.setValue(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
|
||||
props.setProperty(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
|
||||
consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
|
||||
props.setProperty(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
|
||||
consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withOptimisticConsistencyGuardSleepTimeMs(long sleepTimeMs) {
|
||||
props.setProperty(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
|
||||
consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEnableOptimisticConsistencyGuard(boolean enableOptimisticConsistencyGuard) {
|
||||
props.setProperty(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD, String.valueOf(enableOptimisticConsistencyGuard));
|
||||
consistencyGuardConfig.setValue(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP, String.valueOf(enableOptimisticConsistencyGuard));
|
||||
return this;
|
||||
}
|
||||
|
||||
public ConsistencyGuardConfig build() {
|
||||
setDefaultOnCondition(props, !props.containsKey(CONSISTENCY_CHECK_ENABLED_PROP), CONSISTENCY_CHECK_ENABLED_PROP,
|
||||
DEFAULT_CONSISTENCY_CHECK_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP),
|
||||
MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_CONSISTENCY_CHECKS_PROP), MAX_CONSISTENCY_CHECKS_PROP,
|
||||
String.valueOf(DEFAULT_MAX_CONSISTENCY_CHECKS));
|
||||
setDefaultOnCondition(props, !props.containsKey(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP),
|
||||
OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP));
|
||||
setDefaultOnCondition(props, !props.containsKey(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD),
|
||||
ENABLE_OPTIMISTIC_CONSISTENCY_GUARD,
|
||||
String.valueOf(DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD));
|
||||
return new ConsistencyGuardConfig(props);
|
||||
consistencyGuardConfig.setDefaults(ConsistencyGuardConfig.class.getName());
|
||||
return consistencyGuardConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -130,7 +130,7 @@ public class FSUtils {
|
||||
// TODO: this should be removed
|
||||
public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
|
||||
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime,
|
||||
HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
|
||||
HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
|
||||
}
|
||||
|
||||
public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
|
||||
@@ -142,7 +142,8 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
|
||||
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.DEFAULT_BASE_FILE_FORMAT.getFileExtension());
|
||||
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP
|
||||
.defaultValue().getFileExtension());
|
||||
}
|
||||
|
||||
public static String getCommitFromCommitFile(String commitFileName) {
|
||||
@@ -259,7 +260,7 @@ public class FSUtils {
|
||||
.withAssumeDatePartitioning(assumeDatePartitioning)
|
||||
.build();
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
|
||||
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
return tableMetadata.getAllPartitionPaths();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error fetching partition paths from metadata table", e);
|
||||
@@ -269,7 +270,7 @@ public class FSUtils {
|
||||
public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
||||
String basePathStr) {
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
|
||||
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
return tableMetadata.getAllPartitionPaths();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error fetching partition paths from metadata table", e);
|
||||
@@ -279,7 +280,7 @@ public class FSUtils {
|
||||
public static FileStatus[] getFilesInPartition(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
||||
String basePathStr, Path partitionPath) {
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext,
|
||||
metadataConfig, basePathStr, FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
|
||||
metadataConfig, basePathStr, FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
return tableMetadata.getAllFilesInPartition(partitionPath);
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error get files in partition: " + partitionPath, e);
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
@@ -55,6 +56,7 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
if (recordBytes.length == 0) {
|
||||
return Option.empty();
|
||||
}
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
GenericRecord incomingRecord = bytesToAvro(recordBytes, schema);
|
||||
|
||||
// Null check is needed here to support schema evolution. The record in storage may be from old schema where
|
||||
@@ -66,7 +68,8 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
/*
|
||||
* We reached a point where the value is disk is older than the incoming record.
|
||||
*/
|
||||
eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, properties.getProperty(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP), true));
|
||||
eventTime = Option.ofNullable(getNestedFieldVal(incomingRecord, hoodieConfig
|
||||
.getString(HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY), true));
|
||||
|
||||
/*
|
||||
* Now check if the incoming record is a delete record.
|
||||
@@ -99,9 +102,9 @@ public class DefaultHoodieRecordPayload extends OverwriteWithLatestAvroPayload {
|
||||
* and need to be dealt with separately.
|
||||
*/
|
||||
Object persistedOrderingVal = getNestedFieldVal((GenericRecord) currentValue,
|
||||
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), true);
|
||||
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), true);
|
||||
Comparable incomingOrderingVal = (Comparable) getNestedFieldVal((GenericRecord) incomingRecord,
|
||||
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP), false);
|
||||
properties.getProperty(HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY), false);
|
||||
return persistedOrderingVal == null || ((Comparable) persistedOrderingVal).compareTo(incomingOrderingVal) <= 0;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -30,14 +30,12 @@ public class HoodiePayloadProps {
|
||||
*
|
||||
* @see DefaultHoodieRecordPayload
|
||||
*/
|
||||
public static final String PAYLOAD_ORDERING_FIELD_PROP = "hoodie.payload.ordering.field";
|
||||
public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";
|
||||
public static final String PAYLOAD_ORDERING_FIELD_PROP_KEY = "hoodie.payload.ordering.field";
|
||||
|
||||
/**
|
||||
* Property for payload event time field; to be used to extract source event time info.
|
||||
*
|
||||
* @see DefaultHoodieRecordPayload
|
||||
*/
|
||||
public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = "hoodie.payload.event.time.field";
|
||||
public static String DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL = "ts";
|
||||
public static final String PAYLOAD_EVENT_TIME_FIELD_PROP_KEY = "hoodie.payload.event.time.field";
|
||||
}
|
||||
|
||||
@@ -18,10 +18,10 @@
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import java.util.Arrays;
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
import org.apache.hudi.common.bootstrap.index.NoOpBootstrapIndex;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.model.HoodieFileFormat;
|
||||
import org.apache.hudi.common.model.HoodieTableType;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
@@ -30,6 +30,7 @@ import org.apache.hudi.common.util.Option;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
@@ -39,6 +40,7 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.Serializable;
|
||||
import java.util.Arrays;
|
||||
import java.util.Date;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
@@ -51,54 +53,104 @@ import java.util.stream.Collectors;
|
||||
* @see HoodieTableMetaClient
|
||||
* @since 0.3.0
|
||||
*/
|
||||
public class HoodieTableConfig implements Serializable {
|
||||
public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(HoodieTableConfig.class);
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
public static final String HOODIE_TABLE_NAME_PROP_NAME = "hoodie.table.name";
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = "hoodie.table.type";
|
||||
public static final String HOODIE_TABLE_VERSION_PROP_NAME = "hoodie.table.version";
|
||||
public static final String HOODIE_TABLE_PRECOMBINE_FIELD = "hoodie.table.precombine.field";
|
||||
public static final String HOODIE_TABLE_PARTITION_COLUMNS = "hoodie.table.partition.columns";
|
||||
public static final String HOODIE_TABLE_RECORDKEY_FIELDS = "hoodie.table.recordkey.fields";
|
||||
public static final String HOODIE_TABLE_CREATE_SCHEMA = "hoodie.table.create.schema";
|
||||
|
||||
@Deprecated
|
||||
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
|
||||
@Deprecated
|
||||
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
|
||||
public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = "hoodie.table.base.file.format";
|
||||
public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = "hoodie.table.log.file.format";
|
||||
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = "hoodie.timeline.layout.version";
|
||||
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = "hoodie.compaction.payload.class";
|
||||
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = "hoodie.archivelog.folder";
|
||||
public static final String HOODIE_BOOTSTRAP_INDEX_ENABLE = "hoodie.bootstrap.index.enable";
|
||||
public static final String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = "hoodie.bootstrap.index.class";
|
||||
public static final String HOODIE_BOOTSTRAP_BASE_PATH = "hoodie.bootstrap.base.path";
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_NAME_PROP = ConfigProperty
|
||||
.key("hoodie.table.name")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs.");
|
||||
|
||||
public static final ConfigProperty<HoodieTableType> HOODIE_TABLE_TYPE_PROP = ConfigProperty
|
||||
.key("hoodie.table.type")
|
||||
.defaultValue(HoodieTableType.COPY_ON_WRITE)
|
||||
.withDocumentation("The table type for the underlying data, for this write. This can’t change between writes.");
|
||||
|
||||
public static final ConfigProperty<HoodieTableVersion> HOODIE_TABLE_VERSION_PROP = ConfigProperty
|
||||
.key("hoodie.table.version")
|
||||
.defaultValue(HoodieTableVersion.ZERO)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_PRECOMBINE_FIELD_PROP = ConfigProperty
|
||||
.key("hoodie.table.precombine.field")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Field used in preCombining before actual write. When two records have the same key value, "
|
||||
+ "we will pick the one with the largest value for the precombine field, determined by Object.compareTo(..)");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_PARTITION_COLUMNS_PROP = ConfigProperty
|
||||
.key("hoodie.table.partition.columns")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_RECORDKEY_FIELDS = ConfigProperty
|
||||
.key("hoodie.table.recordkey.fields")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_CREATE_SCHEMA = ConfigProperty
|
||||
.key("hoodie.table.create.schema")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty
|
||||
.key("hoodie.table.base.file.format")
|
||||
.defaultValue(HoodieFileFormat.PARQUET)
|
||||
.withAlternatives("hoodie.table.ro.file.format")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_LOG_FILE_FORMAT_PROP = ConfigProperty
|
||||
.key("hoodie.table.log.file.format")
|
||||
.defaultValue(HoodieFileFormat.HOODIE_LOG)
|
||||
.withAlternatives("hoodie.table.rt.file.format")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TIMELINE_LAYOUT_VERSION_PROP = ConfigProperty
|
||||
.key("hoodie.timeline.layout.version")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_PAYLOAD_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.payload.class")
|
||||
.defaultValue(OverwriteWithLatestAvroPayload.class.getName())
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_ARCHIVELOG_FOLDER_PROP = ConfigProperty
|
||||
.key("hoodie.archivelog.folder")
|
||||
.defaultValue("archived")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.enable")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.class")
|
||||
.defaultValue(HFileBootstrapIndex.class.getName())
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_BASE_PATH_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.base.path")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table");
|
||||
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = HoodieTableType.COPY_ON_WRITE;
|
||||
public static final HoodieTableVersion DEFAULT_TABLE_VERSION = HoodieTableVersion.ZERO;
|
||||
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = HoodieFileFormat.PARQUET;
|
||||
public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = HoodieFileFormat.HOODIE_LOG;
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
|
||||
public static final String NO_OP_BOOTSTRAP_INDEX_CLASS = NoOpBootstrapIndex.class.getName();
|
||||
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
|
||||
public static final String DEFAULT_ARCHIVELOG_FOLDER = "archived";
|
||||
|
||||
private Properties props;
|
||||
|
||||
public HoodieTableConfig(FileSystem fs, String metaPath, String payloadClassName) {
|
||||
Properties props = new Properties();
|
||||
super();
|
||||
Path propertyPath = new Path(metaPath, HOODIE_PROPERTIES_FILE);
|
||||
LOG.info("Loading table properties from " + propertyPath);
|
||||
try {
|
||||
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
|
||||
props.load(inputStream);
|
||||
}
|
||||
if (props.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME) && payloadClassName != null
|
||||
&& !props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME).equals(payloadClassName)) {
|
||||
props.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
|
||||
if (contains(HOODIE_PAYLOAD_CLASS_PROP) && payloadClassName != null
|
||||
&& !getString(HOODIE_PAYLOAD_CLASS_PROP).equals(payloadClassName)) {
|
||||
setValue(HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||
props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
@@ -106,21 +158,16 @@ public class HoodieTableConfig implements Serializable {
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
|
||||
}
|
||||
this.props = props;
|
||||
ValidationUtils.checkArgument(props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME) && props.containsKey(HOODIE_TABLE_NAME_PROP_NAME),
|
||||
ValidationUtils.checkArgument(contains(HOODIE_TABLE_TYPE_PROP) && contains(HOODIE_TABLE_NAME_PROP),
|
||||
"hoodie.properties file seems invalid. Please check for left over `.updated` files if any, manually copy it to hoodie.properties and retry");
|
||||
}
|
||||
|
||||
public HoodieTableConfig(Properties props) {
|
||||
this.props = props;
|
||||
}
|
||||
|
||||
/**
|
||||
* For serailizing and de-serializing.
|
||||
* For serializing and de-serializing.
|
||||
*
|
||||
* @deprecated
|
||||
*/
|
||||
public HoodieTableConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -131,30 +178,26 @@ public class HoodieTableConfig implements Serializable {
|
||||
if (!fs.exists(metadataFolder)) {
|
||||
fs.mkdirs(metadataFolder);
|
||||
}
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||
if (!properties.containsKey(HOODIE_TABLE_NAME_PROP_NAME)) {
|
||||
throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP_NAME + " property needs to be specified");
|
||||
if (!hoodieConfig.contains(HOODIE_TABLE_NAME_PROP)) {
|
||||
throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP.key() + " property needs to be specified");
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_TABLE_TYPE_PROP_NAME, DEFAULT_TABLE_TYPE.name());
|
||||
hoodieConfig.setDefaultValue(HOODIE_TABLE_TYPE_PROP);
|
||||
if (hoodieConfig.getString(HOODIE_TABLE_TYPE_PROP).equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
hoodieConfig.setDefaultValue(HOODIE_PAYLOAD_CLASS_PROP);
|
||||
}
|
||||
if (properties.getProperty(HOODIE_TABLE_TYPE_PROP_NAME).equals(HoodieTableType.MERGE_ON_READ.name())
|
||||
&& !properties.containsKey(HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS);
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
|
||||
properties.setProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
|
||||
}
|
||||
if (!properties.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)) {
|
||||
hoodieConfig.setDefaultValue(HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
if (!hoodieConfig.contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
|
||||
// Use latest Version as default unless forced by client
|
||||
properties.setProperty(HOODIE_TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
|
||||
hoodieConfig.setValue(HOODIE_TIMELINE_LAYOUT_VERSION_PROP, TimelineLayoutVersion.CURR_VERSION.toString());
|
||||
}
|
||||
if (properties.containsKey(HOODIE_BOOTSTRAP_BASE_PATH) && !properties.containsKey(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
|
||||
if (hoodieConfig.contains(HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
|
||||
// Use the default bootstrap index class.
|
||||
properties.setProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, getDefaultBootstrapIndexClass(properties));
|
||||
hoodieConfig.setDefaultValue(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(properties));
|
||||
}
|
||||
properties.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -162,15 +205,12 @@ public class HoodieTableConfig implements Serializable {
|
||||
* Read the table type from the table properties and if not found, return the default.
|
||||
*/
|
||||
public HoodieTableType getTableType() {
|
||||
if (props.containsKey(HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
return HoodieTableType.valueOf(props.getProperty(HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_TABLE_TYPE;
|
||||
return HoodieTableType.valueOf(getStringOrDefault(HOODIE_TABLE_TYPE_PROP));
|
||||
}
|
||||
|
||||
public Option<TimelineLayoutVersion> getTimelineLayoutVersion() {
|
||||
return props.containsKey(HOODIE_TIMELINE_LAYOUT_VERSION)
|
||||
? Option.of(new TimelineLayoutVersion(Integer.valueOf(props.getProperty(HOODIE_TIMELINE_LAYOUT_VERSION))))
|
||||
return contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)
|
||||
? Option.of(new TimelineLayoutVersion(getInt(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)))
|
||||
: Option.empty();
|
||||
}
|
||||
|
||||
@@ -178,13 +218,13 @@ public class HoodieTableConfig implements Serializable {
|
||||
* @return the hoodie.table.version from hoodie.properties file.
|
||||
*/
|
||||
public HoodieTableVersion getTableVersion() {
|
||||
return props.containsKey(HOODIE_TABLE_VERSION_PROP_NAME)
|
||||
? HoodieTableVersion.versionFromCode(Integer.parseInt(props.getProperty(HOODIE_TABLE_VERSION_PROP_NAME)))
|
||||
: DEFAULT_TABLE_VERSION;
|
||||
return contains(HOODIE_TABLE_VERSION_PROP)
|
||||
? HoodieTableVersion.versionFromCode(getInt(HOODIE_TABLE_VERSION_PROP))
|
||||
: HOODIE_TABLE_VERSION_PROP.defaultValue();
|
||||
}
|
||||
|
||||
public void setTableVersion(HoodieTableVersion tableVersion) {
|
||||
props.put(HOODIE_TABLE_VERSION_PROP_NAME, Integer.toString(tableVersion.versionCode()));
|
||||
setValue(HOODIE_TABLE_VERSION_PROP, Integer.toString(tableVersion.versionCode()));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -193,17 +233,17 @@ public class HoodieTableConfig implements Serializable {
|
||||
public String getPayloadClass() {
|
||||
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
|
||||
// change to org.apache.hudi
|
||||
return props.getProperty(HOODIE_PAYLOAD_CLASS_PROP_NAME, DEFAULT_PAYLOAD_CLASS).replace("com.uber.hoodie",
|
||||
return getStringOrDefault(HOODIE_PAYLOAD_CLASS_PROP).replace("com.uber.hoodie",
|
||||
"org.apache.hudi");
|
||||
}
|
||||
|
||||
public String getPreCombineField() {
|
||||
return props.getProperty(HOODIE_TABLE_PRECOMBINE_FIELD);
|
||||
return getString(HOODIE_TABLE_PRECOMBINE_FIELD_PROP);
|
||||
}
|
||||
|
||||
public Option<String[]> getPartitionColumns() {
|
||||
if (props.containsKey(HOODIE_TABLE_PARTITION_COLUMNS)) {
|
||||
return Option.of(Arrays.stream(props.getProperty(HOODIE_TABLE_PARTITION_COLUMNS).split(","))
|
||||
if (contains(HOODIE_TABLE_PARTITION_COLUMNS_PROP)) {
|
||||
return Option.of(Arrays.stream(getString(HOODIE_TABLE_PARTITION_COLUMNS_PROP).split(","))
|
||||
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
|
||||
}
|
||||
return Option.empty();
|
||||
@@ -215,24 +255,24 @@ public class HoodieTableConfig implements Serializable {
|
||||
public String getBootstrapIndexClass() {
|
||||
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
|
||||
// change to org.apache.hudi
|
||||
return props.getProperty(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, getDefaultBootstrapIndexClass(props));
|
||||
return getStringOrDefault(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(props));
|
||||
}
|
||||
|
||||
public static String getDefaultBootstrapIndexClass(Properties props) {
|
||||
String defaultClass = DEFAULT_BOOTSTRAP_INDEX_CLASS;
|
||||
if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE))) {
|
||||
String defaultClass = HOODIE_BOOTSTRAP_INDEX_CLASS_PROP.defaultValue();
|
||||
if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key()))) {
|
||||
defaultClass = NO_OP_BOOTSTRAP_INDEX_CLASS;
|
||||
}
|
||||
return defaultClass;
|
||||
}
|
||||
|
||||
public Option<String> getBootstrapBasePath() {
|
||||
return Option.ofNullable(props.getProperty(HOODIE_BOOTSTRAP_BASE_PATH));
|
||||
return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP));
|
||||
}
|
||||
|
||||
public Option<Schema> getTableCreateSchema() {
|
||||
if (props.containsKey(HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
return Option.of(new Schema.Parser().parse(props.getProperty(HOODIE_TABLE_CREATE_SCHEMA)));
|
||||
if (contains(HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
return Option.of(new Schema.Parser().parse(getString(HOODIE_TABLE_CREATE_SCHEMA)));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
@@ -242,7 +282,7 @@ public class HoodieTableConfig implements Serializable {
|
||||
* Read the table name.
|
||||
*/
|
||||
public String getTableName() {
|
||||
return props.getProperty(HOODIE_TABLE_NAME_PROP_NAME);
|
||||
return getString(HOODIE_TABLE_NAME_PROP);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -251,13 +291,7 @@ public class HoodieTableConfig implements Serializable {
|
||||
* @return HoodieFileFormat for the base file Storage format
|
||||
*/
|
||||
public HoodieFileFormat getBaseFileFormat() {
|
||||
if (props.containsKey(HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_BASE_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
if (props.containsKey(HOODIE_RO_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RO_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_BASE_FILE_FORMAT;
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_BASE_FILE_FORMAT_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -266,28 +300,18 @@ public class HoodieTableConfig implements Serializable {
|
||||
* @return HoodieFileFormat for the log Storage format
|
||||
*/
|
||||
public HoodieFileFormat getLogFileFormat() {
|
||||
if (props.containsKey(HOODIE_LOG_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_LOG_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
if (props.containsKey(HOODIE_RT_FILE_FORMAT_PROP_NAME)) {
|
||||
return HoodieFileFormat.valueOf(props.getProperty(HOODIE_RT_FILE_FORMAT_PROP_NAME));
|
||||
}
|
||||
return DEFAULT_LOG_FILE_FORMAT;
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_LOG_FILE_FORMAT_PROP));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the relative path of archive log folder under metafolder, for this table.
|
||||
*/
|
||||
public String getArchivelogFolder() {
|
||||
return props.getProperty(HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, DEFAULT_ARCHIVELOG_FOLDER);
|
||||
return getStringOrDefault(HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
}
|
||||
|
||||
public Map<String, String> getProps() {
|
||||
public Map<String, String> propsMap() {
|
||||
return props.entrySet().stream()
|
||||
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
|
||||
}
|
||||
|
||||
public Properties getProperties() {
|
||||
return props;
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.common.table;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
|
||||
import org.apache.hudi.common.fs.FSUtils;
|
||||
@@ -225,7 +226,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public String getArchivePath() {
|
||||
String archiveFolder = tableConfig.getArchivelogFolder();
|
||||
if (archiveFolder.equals(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER)) {
|
||||
if (archiveFolder.equals(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())) {
|
||||
return getMetaPath();
|
||||
} else {
|
||||
return getMetaPath() + "/" + archiveFolder;
|
||||
@@ -330,8 +331,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
}
|
||||
|
||||
// if anything other than default archive log folder is specified, create that too
|
||||
String archiveLogPropVal = props.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME,
|
||||
HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
|
||||
String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
if (!StringUtils.isNullOrEmpty(archiveLogPropVal)) {
|
||||
Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
|
||||
if (!fs.exists(archiveLogDir)) {
|
||||
@@ -683,47 +683,47 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
}
|
||||
|
||||
public PropertyBuilder fromProperties(Properties properties) {
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME)) {
|
||||
setTableName(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME));
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_NAME_PROP)) {
|
||||
setTableName(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_NAME_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME)) {
|
||||
setTableType(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP)) {
|
||||
setTableType(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)) {
|
||||
setArchiveLogFolder(
|
||||
properties.getProperty(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME));
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP)) {
|
||||
setPayloadClassName(
|
||||
properties.getProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME));
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)) {
|
||||
setTimelineLayoutVersion(Integer
|
||||
.parseInt(properties.getProperty(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION)));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
|
||||
setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) {
|
||||
setBaseFileFormat(
|
||||
properties.getProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME));
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP)) {
|
||||
setBootstrapIndexClass(
|
||||
properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME));
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH)) {
|
||||
setBootstrapBasePath(properties.getProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
|
||||
setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD)) {
|
||||
setPreCombineField(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP)) {
|
||||
setPreCombineField(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP)) {
|
||||
setPartitionColumns(
|
||||
properties.getProperty(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS));
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
|
||||
setRecordKeyFields(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
|
||||
setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
|
||||
}
|
||||
if (properties.containsKey(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
setTableCreateSchema(properties.getProperty(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
@@ -732,55 +732,53 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
ValidationUtils.checkArgument(tableType != null, "tableType is null");
|
||||
ValidationUtils.checkArgument(tableName != null, "tableName is null");
|
||||
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP_NAME, tableName);
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP_NAME, tableType.name());
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP_NAME,
|
||||
HoodieTableConfig tableConfig = new HoodieTableConfig();
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_NAME_PROP, tableName);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP, tableType.name());
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP,
|
||||
String.valueOf(HoodieTableVersion.current().versionCode()));
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP_NAME, payloadClassName);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
}
|
||||
|
||||
if (null != tableCreateSchema) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
|
||||
}
|
||||
|
||||
if (!StringUtils.isNullOrEmpty(archiveLogFolder)) {
|
||||
properties.put(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, archiveLogFolder);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP, archiveLogFolder);
|
||||
} else {
|
||||
properties.setProperty(HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER, HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER);
|
||||
tableConfig.setDefaultValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
}
|
||||
|
||||
if (null != timelineLayoutVersion) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION,
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP,
|
||||
String.valueOf(timelineLayoutVersion));
|
||||
}
|
||||
|
||||
if (null != baseFileFormat) {
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP_NAME,
|
||||
baseFileFormat.toUpperCase());
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP, baseFileFormat.toUpperCase());
|
||||
}
|
||||
|
||||
if (null != bootstrapIndexClass) {
|
||||
properties
|
||||
.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME, bootstrapIndexClass);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, bootstrapIndexClass);
|
||||
}
|
||||
|
||||
if (null != bootstrapBasePath) {
|
||||
properties.put(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH, bootstrapBasePath);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP, bootstrapBasePath);
|
||||
}
|
||||
|
||||
if (null != preCombineField) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD, preCombineField);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, preCombineField);
|
||||
}
|
||||
|
||||
if (null != partitionColumns) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS, partitionColumns);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PARTITION_COLUMNS_PROP, partitionColumns);
|
||||
}
|
||||
if (null != recordKeyFields) {
|
||||
properties.put(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
|
||||
}
|
||||
return properties;
|
||||
return tableConfig.getProps();
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.common.table.view;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.util.ValidationUtils;
|
||||
|
||||
import java.io.File;
|
||||
@@ -29,136 +30,156 @@ import java.util.Properties;
|
||||
/**
|
||||
* File System View Storage Configurations.
|
||||
*/
|
||||
public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
|
||||
public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
|
||||
// Property Names
|
||||
public static final String FILESYSTEM_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.type";
|
||||
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
|
||||
public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_VIEW_STORAGE_TYPE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "hoodie.filesystem.view.incr.timeline.sync.enable";
|
||||
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = "false";
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.incr.timeline.sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = "hoodie.filesystem.view.secondary.type";
|
||||
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY;
|
||||
public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.secondary.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_HOST = "hoodie.filesystem.view.remote.host";
|
||||
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = "localhost";
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_REMOTE_HOST = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.host")
|
||||
.defaultValue("localhost")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_PORT = "hoodie.filesystem.view.remote.port";
|
||||
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = 26754;
|
||||
public static final ConfigProperty<Integer> FILESYSTEM_VIEW_REMOTE_PORT = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.port")
|
||||
.defaultValue(26754)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = "hoodie.filesystem.view.spillable.dir";
|
||||
public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/";
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_SPILLABLE_DIR = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.dir")
|
||||
.defaultValue("/tmp/view_map/")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = "hoodie.filesystem.view.spillable.mem";
|
||||
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB
|
||||
public static final ConfigProperty<Long> FILESYSTEM_VIEW_SPILLABLE_MEM = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.mem")
|
||||
.defaultValue(100 * 1024 * 1024L) // 100 MB
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION =
|
||||
"hoodie.filesystem.view.spillable.compaction.mem.fraction";
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01;
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.compaction.mem.fraction")
|
||||
.defaultValue(0.8)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION =
|
||||
"hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction";
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
|
||||
.defaultValue(0.05)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION =
|
||||
"hoodie.filesystem.view.spillable.replaced.mem.fraction";
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01;
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.replaced.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION =
|
||||
"hoodie.filesystem.view.spillable.clustering.mem.fraction";
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = 0.01;
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.clustering.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("");
|
||||
|
||||
private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path";
|
||||
public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb";
|
||||
public static final ConfigProperty<String> ROCKSDB_BASE_PATH_PROP = ConfigProperty
|
||||
.key("hoodie.filesystem.view.rocksdb.base.path")
|
||||
.defaultValue("/tmp/hoodie_timeline_rocksdb")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS =
|
||||
"hoodie.filesystem.view.remote.timeout.secs";
|
||||
public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = 5 * 60; // 5 min
|
||||
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05;
|
||||
public static final ConfigProperty<Integer> FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.timeout.secs")
|
||||
.defaultValue(5 * 60) // 5 min
|
||||
.withDocumentation("");
|
||||
|
||||
/**
|
||||
* Configs to control whether backup needs to be configured if clients were not able to reach
|
||||
* timeline service.
|
||||
*/
|
||||
public static final String REMOTE_BACKUP_VIEW_HANDLER_ENABLE =
|
||||
"hoodie.filesystem.remote.backup.view.enable";
|
||||
// Need to be disabled only for tests.
|
||||
public static final String DEFAULT_REMOTE_BACKUP_VIEW_HANDLER_ENABLE = "true";
|
||||
public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_HANDLER_ENABLE = ConfigProperty
|
||||
.key("hoodie.filesystem.remote.backup.view.enable")
|
||||
.defaultValue("true") // Need to be disabled only for tests.
|
||||
.withDocumentation("");
|
||||
|
||||
public static FileSystemViewStorageConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
}
|
||||
|
||||
private FileSystemViewStorageConfig(Properties props) {
|
||||
super(props);
|
||||
private FileSystemViewStorageConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
}
|
||||
|
||||
public boolean isIncrementalTimelineSyncEnabled() {
|
||||
return Boolean.parseBoolean(props.getProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE));
|
||||
return getBoolean(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
|
||||
}
|
||||
|
||||
public String getRemoteViewServerHost() {
|
||||
return props.getProperty(FILESYSTEM_VIEW_REMOTE_HOST);
|
||||
return getString(FILESYSTEM_VIEW_REMOTE_HOST);
|
||||
}
|
||||
|
||||
public Integer getRemoteViewServerPort() {
|
||||
return Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT));
|
||||
return getInt(FILESYSTEM_VIEW_REMOTE_PORT);
|
||||
}
|
||||
|
||||
public Integer getRemoteTimelineClientTimeoutSecs() {
|
||||
return Integer.parseInt(props.getProperty(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS));
|
||||
return getInt(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
|
||||
}
|
||||
|
||||
public long getMaxMemoryForFileGroupMap() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForPendingCompaction() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION)))
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForBootstrapBaseFile() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
long reservedForExternalDataFile =
|
||||
new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION)))
|
||||
new Double(totalMemory * getDouble(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION))
|
||||
.longValue();
|
||||
return reservedForExternalDataFile;
|
||||
}
|
||||
|
||||
public long getMaxMemoryForReplacedFileGroups() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION)))
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForPendingClusteringFileGroups() {
|
||||
long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM));
|
||||
return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION)))
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public String getSpillableDir() {
|
||||
return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR);
|
||||
return getString(FILESYSTEM_VIEW_SPILLABLE_DIR);
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getSecondaryStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
}
|
||||
|
||||
public boolean shouldEnableBackupForRemoteFileSystemView() {
|
||||
return Boolean.parseBoolean(props.getProperty(REMOTE_BACKUP_VIEW_HANDLER_ENABLE));
|
||||
return getBoolean(REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
|
||||
}
|
||||
|
||||
public String getRocksdbBasePath() {
|
||||
return props.getProperty(ROCKSDB_BASE_PATH_PROP);
|
||||
return getString(ROCKSDB_BASE_PATH_PROP);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -166,118 +187,87 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig {
|
||||
*/
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final FileSystemViewStorageConfig fileSystemViewStorageConfig = new FileSystemViewStorageConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
props.load(reader);
|
||||
fileSystemViewStorageConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.fileSystemViewStorageConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withStorageType(FileSystemViewStorageType storageType) {
|
||||
props.setProperty(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
|
||||
props.setProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
|
||||
props.setProperty(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerHost(String remoteServerHost) {
|
||||
props.setProperty(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerPort(Integer remoteServerPort) {
|
||||
props.setProperty(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryForView(Long maxMemoryForView) {
|
||||
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteTimelineClientTimeoutSecs(Long timelineClientTimeoutSecs) {
|
||||
props.setProperty(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
|
||||
props.setProperty(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) {
|
||||
props.setProperty(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBaseStoreDir(String baseStorePath) {
|
||||
props.setProperty(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRocksDBPath(String basePath) {
|
||||
props.setProperty(ROCKSDB_BASE_PATH_PROP, basePath);
|
||||
fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEnableBackupForRemoteFileSystemView(boolean enable) {
|
||||
props.setProperty(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig build() {
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_STORAGE_TYPE), FILESYSTEM_VIEW_STORAGE_TYPE,
|
||||
DEFAULT_VIEW_STORAGE_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE),
|
||||
FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE),
|
||||
FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, DEFAULT_SECONDARY_VIEW_STORAGE_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_HOST), FILESYSTEM_VIEW_REMOTE_HOST,
|
||||
DEFUALT_REMOTE_VIEW_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REMOTE_PORT), FILESYSTEM_VIEW_REMOTE_PORT,
|
||||
DEFAULT_REMOTE_VIEW_SERVER_PORT.toString());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_DIR), FILESYSTEM_VIEW_SPILLABLE_DIR,
|
||||
DEFAULT_VIEW_SPILLABLE_DIR);
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_SPILLABLE_MEM), FILESYSTEM_VIEW_SPILLABLE_MEM,
|
||||
DEFAULT_MAX_MEMORY_FOR_VIEW.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS),
|
||||
FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION),
|
||||
FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION),
|
||||
FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION),
|
||||
FILESYSTEM_VIEW_REPLACED_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS.toString());
|
||||
setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION),
|
||||
FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS.toString());
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP,
|
||||
DEFAULT_ROCKSDB_BASE_PATH);
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(REMOTE_BACKUP_VIEW_HANDLER_ENABLE),
|
||||
REMOTE_BACKUP_VIEW_HANDLER_ENABLE, DEFAULT_REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
|
||||
|
||||
fileSystemViewStorageConfig.setDefaults(FileSystemViewStorageConfig.class.getName());
|
||||
// Validations
|
||||
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
FileSystemViewStorageType.valueOf(props.getProperty(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
ValidationUtils.checkArgument(Integer.parseInt(props.getProperty(FILESYSTEM_VIEW_REMOTE_PORT)) > 0);
|
||||
return new FileSystemViewStorageConfig(props);
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
ValidationUtils.checkArgument(fileSystemViewStorageConfig.getInt(FILESYSTEM_VIEW_REMOTE_PORT) > 0);
|
||||
return fileSystemViewStorageConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -51,7 +51,7 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
|
||||
HoodieMetadataConfig metadataConfig) {
|
||||
super(metaClient, visibleActiveTimeline);
|
||||
this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(),
|
||||
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR);
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
Reference in New Issue
Block a user