[HUDI-89] Add configOption & refactor all configs based on that (#2833)
Co-authored-by: Wenning Ding <wenningd@amazon.com>
This commit is contained in:
@@ -33,7 +33,6 @@ import org.apache.log4j.Logger;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
/**
|
||||
* Write commit callback http client.
|
||||
@@ -47,10 +46,10 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
|
||||
private final String apiKey;
|
||||
private final String url;
|
||||
private final CloseableHttpClient client;
|
||||
private Properties props;
|
||||
private HoodieWriteConfig writeConfig;
|
||||
|
||||
public HoodieWriteCommitHttpCallbackClient(HoodieWriteConfig config) {
|
||||
this.props = config.getProps();
|
||||
this.writeConfig = config;
|
||||
this.apiKey = getApiKey();
|
||||
this.url = getUrl();
|
||||
this.client = getClient();
|
||||
@@ -80,11 +79,11 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
|
||||
}
|
||||
|
||||
private String getApiKey() {
|
||||
return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
|
||||
return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_API_KEY);
|
||||
}
|
||||
|
||||
private String getUrl() {
|
||||
return props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_URL_PROP);
|
||||
return writeConfig.getString(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_URL_PROP);
|
||||
}
|
||||
|
||||
private CloseableHttpClient getClient() {
|
||||
@@ -98,7 +97,7 @@ public class HoodieWriteCommitHttpCallbackClient implements Closeable {
|
||||
}
|
||||
|
||||
private Integer getHttpTimeoutSeconds() {
|
||||
return Integer.parseInt(props.getProperty(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_SECONDS));
|
||||
return writeConfig.getInt(HoodieWriteCommitCallbackConfig.CALLBACK_HTTP_TIMEOUT_SECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -39,7 +39,7 @@ public class HoodieCommitCallbackFactory {
|
||||
return (HoodieWriteCommitCallback) instance;
|
||||
} else {
|
||||
throw new HoodieCommitCallbackException(String.format("The value of the config option %s can not be null or "
|
||||
+ "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP));
|
||||
+ "empty", HoodieWriteCommitCallbackConfig.CALLBACK_CLASS_PROP.key()));
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -441,19 +441,19 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
|
||||
// Do an inline compaction if enabled
|
||||
if (config.inlineCompactionEnabled()) {
|
||||
runAnyPendingCompactions(table);
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "true");
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP.key(), "true");
|
||||
inlineCompact(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP, "false");
|
||||
metadata.addMetadata(HoodieCompactionConfig.INLINE_COMPACT_PROP.key(), "false");
|
||||
}
|
||||
|
||||
// Do an inline clustering if enabled
|
||||
if (config.inlineClusteringEnabled()) {
|
||||
runAnyPendingClustering(table);
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "true");
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), "true");
|
||||
inlineCluster(extraMetadata);
|
||||
} else {
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP, "false");
|
||||
metadata.addMetadata(HoodieClusteringConfig.INLINE_CLUSTERING_PROP.key(), "false");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,9 +18,6 @@
|
||||
|
||||
package org.apache.hudi.client.transaction.lock;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
|
||||
import java.io.Serializable;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
@@ -36,6 +33,9 @@ import org.apache.hudi.exception.HoodieLockException;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
|
||||
/**
|
||||
* This class wraps implementations of {@link LockProvider} and provides an easy way to manage the lifecycle of a lock.
|
||||
*/
|
||||
@@ -61,8 +61,8 @@ public class LockManager implements Serializable, AutoCloseable {
|
||||
LockProvider lockProvider = getLockProvider();
|
||||
int retryCount = 0;
|
||||
boolean acquired = false;
|
||||
int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP);
|
||||
long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP);
|
||||
int retries = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY);
|
||||
long waitTimeInMs = lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY);
|
||||
while (retryCount <= retries) {
|
||||
try {
|
||||
acquired = lockProvider.tryLock(writeConfig.getLockAcquireWaitTimeoutInMs(), TimeUnit.MILLISECONDS);
|
||||
|
||||
@@ -38,14 +38,14 @@ import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
|
||||
|
||||
/**
|
||||
* A zookeeper based lock. This {@link LockProvider} implementation allows to lock table operations
|
||||
@@ -64,11 +64,11 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
|
||||
checkRequiredProps(lockConfiguration);
|
||||
this.lockConfiguration = lockConfiguration;
|
||||
this.curatorFrameworkClient = CuratorFrameworkFactory.builder()
|
||||
.connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP))
|
||||
.retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP), lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP)))
|
||||
.sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP, DEFAULT_ZK_SESSION_TIMEOUT_MS))
|
||||
.connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
|
||||
.connectString(lockConfiguration.getConfig().getString(ZK_CONNECT_URL_PROP_KEY))
|
||||
.retryPolicy(new BoundedExponentialBackoffRetry(lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY),
|
||||
lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY), lockConfiguration.getConfig().getInteger(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY)))
|
||||
.sessionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_SESSION_TIMEOUT_MS_PROP_KEY, DEFAULT_ZK_SESSION_TIMEOUT_MS))
|
||||
.connectionTimeoutMs(lockConfiguration.getConfig().getInteger(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY, DEFAULT_ZK_CONNECTION_TIMEOUT_MS))
|
||||
.build();
|
||||
this.curatorFrameworkClient.start();
|
||||
}
|
||||
@@ -136,8 +136,8 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
|
||||
private void acquireLock(long time, TimeUnit unit) throws Exception {
|
||||
ValidationUtils.checkArgument(this.lock == null, generateLogStatement(LockState.ALREADY_ACQUIRED, generateLogSuffixString()));
|
||||
InterProcessMutex newLock = new InterProcessMutex(
|
||||
this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP) + "/"
|
||||
+ this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP));
|
||||
this.curatorFrameworkClient, lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP_KEY) + "/"
|
||||
+ this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP_KEY));
|
||||
boolean acquired = newLock.acquire(time, unit);
|
||||
if (!acquired) {
|
||||
throw new HoodieLockException(generateLogStatement(LockState.FAILED_TO_ACQUIRE, generateLogSuffixString()));
|
||||
@@ -150,16 +150,16 @@ public class ZookeeperBasedLockProvider implements LockProvider<InterProcessMute
|
||||
}
|
||||
|
||||
private void checkRequiredProps(final LockConfiguration config) {
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECT_URL_PROP_KEY) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_BASE_PATH_PROP_KEY) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_SESSION_TIMEOUT_MS_PROP_KEY) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY) != null);
|
||||
ValidationUtils.checkArgument(config.getConfig().getString(ZK_LOCK_KEY_PROP_KEY) != null);
|
||||
}
|
||||
|
||||
private String generateLogSuffixString() {
|
||||
String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP);
|
||||
String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP);
|
||||
String zkBasePath = this.lockConfiguration.getConfig().getString(ZK_BASE_PATH_PROP_KEY);
|
||||
String lockKey = this.lockConfiguration.getConfig().getString(ZK_LOCK_KEY_PROP_KEY);
|
||||
return StringUtils.join("ZkBasePath = ", zkBasePath, ", lock key = ", lockKey);
|
||||
}
|
||||
|
||||
|
||||
@@ -22,7 +22,8 @@ import org.apache.hudi.client.bootstrap.BootstrapMode;
|
||||
import org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector;
|
||||
import org.apache.hudi.client.bootstrap.translator.IdentityBootstrapPartitionPathTranslator;
|
||||
import org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex;
|
||||
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.table.HoodieTableConfig;
|
||||
|
||||
import java.io.File;
|
||||
@@ -33,33 +34,66 @@ import java.util.Properties;
|
||||
/**
|
||||
* Bootstrap specific configs.
|
||||
*/
|
||||
public class HoodieBootstrapConfig extends DefaultHoodieConfig {
|
||||
public class HoodieBootstrapConfig extends HoodieConfig {
|
||||
|
||||
public static final String BOOTSTRAP_BASE_PATH_PROP = "hoodie.bootstrap.base.path";
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR = "hoodie.bootstrap.mode.selector";
|
||||
public static final String FULL_BOOTSTRAP_INPUT_PROVIDER = "hoodie.bootstrap.full.input.provider";
|
||||
public static final String DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER = "org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider";
|
||||
public static final String BOOTSTRAP_KEYGEN_CLASS = "hoodie.bootstrap.keygen.class";
|
||||
public static final String BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
|
||||
"hoodie.bootstrap.partitionpath.translator.class";
|
||||
public static final String DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS =
|
||||
IdentityBootstrapPartitionPathTranslator.class.getName();
|
||||
public static final ConfigProperty<String> BOOTSTRAP_BASE_PATH_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.base.path")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table");
|
||||
|
||||
public static final String BOOTSTRAP_PARALLELISM = "hoodie.bootstrap.parallelism";
|
||||
public static final String DEFAULT_BOOTSTRAP_PARALLELISM = "1500";
|
||||
public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR = ConfigProperty
|
||||
.key("hoodie.bootstrap.mode.selector")
|
||||
.defaultValue(MetadataOnlyBootstrapModeSelector.class.getCanonicalName())
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Selects the mode in which each file/partition in the bootstrapped dataset gets bootstrapped");
|
||||
|
||||
// Used By BootstrapRegexModeSelector class. When a partition path matches the regex, the corresponding
|
||||
// mode will be used. Otherwise, the alternative mode will be used.
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR_REGEX = "hoodie.bootstrap.mode.selector.regex";
|
||||
public static final String BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = "hoodie.bootstrap.mode.selector.regex.mode";
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX = ".*";
|
||||
public static final String DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = BootstrapMode.METADATA_ONLY.name();
|
||||
public static final ConfigProperty<String> FULL_BOOTSTRAP_INPUT_PROVIDER = ConfigProperty
|
||||
.key("hoodie.bootstrap.full.input.provider")
|
||||
.defaultValue("org.apache.hudi.bootstrap.SparkParquetBootstrapDataProvider")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Class to use for reading the bootstrap dataset partitions/files, for Bootstrap mode FULL_RECORD");
|
||||
|
||||
public static final String BOOTSTRAP_INDEX_CLASS_PROP = "hoodie.bootstrap.index.class";
|
||||
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = HFileBootstrapIndex.class.getName();
|
||||
public static final ConfigProperty<String> BOOTSTRAP_KEYGEN_CLASS = ConfigProperty
|
||||
.key("hoodie.bootstrap.keygen.class")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Key generator implementation to be used for generating keys from the bootstrapped dataset");
|
||||
|
||||
public HoodieBootstrapConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS = ConfigProperty
|
||||
.key("hoodie.bootstrap.partitionpath.translator.class")
|
||||
.defaultValue(IdentityBootstrapPartitionPathTranslator.class.getName())
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Translates the partition paths from the bootstrapped data into how is laid out as a Hudi table.");
|
||||
|
||||
public static final ConfigProperty<String> BOOTSTRAP_PARALLELISM = ConfigProperty
|
||||
.key("hoodie.bootstrap.parallelism")
|
||||
.defaultValue("1500")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Parallelism value to be used to bootstrap data into hudi");
|
||||
|
||||
public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR_REGEX = ConfigProperty
|
||||
.key("hoodie.bootstrap.mode.selector.regex")
|
||||
.defaultValue(".*")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Matches each bootstrap dataset partition against this regex and applies the mode below to it.");
|
||||
|
||||
public static final ConfigProperty<String> BOOTSTRAP_MODE_SELECTOR_REGEX_MODE = ConfigProperty
|
||||
.key("hoodie.bootstrap.mode.selector.regex.mode")
|
||||
.defaultValue(BootstrapMode.METADATA_ONLY.name())
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Bootstrap mode to apply for partition paths, that match regex above. "
|
||||
+ "METADATA_ONLY will generate just skeleton base files with keys/footers, avoiding full cost of rewriting the dataset. "
|
||||
+ "FULL_RECORD will perform a full copy/rewrite of the data as a Hudi table.");
|
||||
|
||||
public static final ConfigProperty<String> BOOTSTRAP_INDEX_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.class")
|
||||
.defaultValue(HFileBootstrapIndex.class.getName())
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
private HoodieBootstrapConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
@@ -68,78 +102,67 @@ public class HoodieBootstrapConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieBootstrapConfig bootstrapConfig = new HoodieBootstrapConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.bootstrapConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withBootstrapBasePath(String basePath) {
|
||||
props.setProperty(BOOTSTRAP_BASE_PATH_PROP, basePath);
|
||||
bootstrapConfig.setValue(BOOTSTRAP_BASE_PATH_PROP, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeSelector(String partitionSelectorClass) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
|
||||
bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR, partitionSelectorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFullBootstrapInputProvider(String partitionSelectorClass) {
|
||||
props.setProperty(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
|
||||
bootstrapConfig.setValue(FULL_BOOTSTRAP_INPUT_PROVIDER, partitionSelectorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapKeyGenClass(String keyGenClass) {
|
||||
props.setProperty(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
|
||||
bootstrapConfig.setValue(BOOTSTRAP_KEYGEN_CLASS, keyGenClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapPartitionPathTranslatorClass(String partitionPathTranslatorClass) {
|
||||
props.setProperty(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
|
||||
bootstrapConfig
|
||||
.setValue(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, partitionPathTranslatorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapParallelism(int parallelism) {
|
||||
props.setProperty(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
|
||||
bootstrapConfig.setValue(BOOTSTRAP_PARALLELISM, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeSelectorRegex(String regex) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
|
||||
bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX, regex);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBootstrapModeForRegexMatch(BootstrapMode modeForRegexMatch) {
|
||||
props.setProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
|
||||
bootstrapConfig.setValue(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, modeForRegexMatch.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.bootstrapConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieBootstrapConfig build() {
|
||||
HoodieBootstrapConfig config = new HoodieBootstrapConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARALLELISM), BOOTSTRAP_PARALLELISM,
|
||||
DEFAULT_BOOTSTRAP_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS),
|
||||
BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS, DEFAULT_BOOTSTRAP_PARTITION_PATH_TRANSLATOR_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR), BOOTSTRAP_MODE_SELECTOR,
|
||||
MetadataOnlyBootstrapModeSelector.class.getCanonicalName());
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX), BOOTSTRAP_MODE_SELECTOR_REGEX,
|
||||
DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX);
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE),
|
||||
BOOTSTRAP_MODE_SELECTOR_REGEX_MODE, DEFAULT_BOOTSTRAP_MODE_SELECTOR_REGEX_MODE);
|
||||
BootstrapMode.valueOf(props.getProperty(BOOTSTRAP_MODE_SELECTOR_REGEX_MODE));
|
||||
setDefaultOnCondition(props, !props.containsKey(BOOTSTRAP_INDEX_CLASS_PROP), BOOTSTRAP_INDEX_CLASS_PROP,
|
||||
HoodieTableConfig.getDefaultBootstrapIndexClass(props));
|
||||
setDefaultOnCondition(props, !props.containsKey(FULL_BOOTSTRAP_INPUT_PROVIDER), FULL_BOOTSTRAP_INPUT_PROVIDER,
|
||||
DEFAULT_FULL_BOOTSTRAP_INPUT_PROVIDER);
|
||||
return config;
|
||||
// TODO: use infer function instead
|
||||
bootstrapConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_PROP, HoodieTableConfig.getDefaultBootstrapIndexClass(
|
||||
bootstrapConfig.getProps()));
|
||||
bootstrapConfig.setDefaults(HoodieBootstrapConfig.class.getName());
|
||||
return bootstrapConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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,64 +29,87 @@ import java.util.Properties;
|
||||
/**
|
||||
* Clustering specific configs.
|
||||
*/
|
||||
public class HoodieClusteringConfig extends DefaultHoodieConfig {
|
||||
public class HoodieClusteringConfig extends HoodieConfig {
|
||||
|
||||
// Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy
|
||||
public static final String CLUSTERING_PLAN_STRATEGY_CLASS = "hoodie.clustering.plan.strategy.class";
|
||||
public static final String DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS =
|
||||
"org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy";
|
||||
public static final ConfigProperty<String> CLUSTERING_PLAN_STRATEGY_CLASS = ConfigProperty
|
||||
.key("hoodie.clustering.plan.strategy.class")
|
||||
.defaultValue("org.apache.hudi.client.clustering.plan.strategy.SparkRecentDaysClusteringPlanStrategy")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Config to provide a strategy class to create ClusteringPlan. Class has to be subclass of ClusteringPlanStrategy");
|
||||
|
||||
// Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy
|
||||
public static final String CLUSTERING_EXECUTION_STRATEGY_CLASS = "hoodie.clustering.execution.strategy.class";
|
||||
public static final String DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS =
|
||||
"org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy";
|
||||
public static final ConfigProperty<String> CLUSTERING_EXECUTION_STRATEGY_CLASS = ConfigProperty
|
||||
.key("hoodie.clustering.execution.strategy.class")
|
||||
.defaultValue("org.apache.hudi.client.clustering.run.strategy.SparkSortAndSizeExecutionStrategy")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Config to provide a strategy class to execute a ClusteringPlan. Class has to be subclass of RunClusteringStrategy");
|
||||
|
||||
// Turn on inline clustering - clustering will be run after write operation is complete.
|
||||
public static final String INLINE_CLUSTERING_PROP = "hoodie.clustering.inline";
|
||||
private static final String DEFAULT_INLINE_CLUSTERING = "false";
|
||||
public static final ConfigProperty<String> INLINE_CLUSTERING_PROP = ConfigProperty
|
||||
.key("hoodie.clustering.inline")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Turn on inline clustering - clustering will be run after write operation is complete");
|
||||
|
||||
public static final ConfigProperty<String> INLINE_CLUSTERING_MAX_COMMIT_PROP = ConfigProperty
|
||||
.key("hoodie.clustering.inline.max.commits")
|
||||
.defaultValue("4")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Config to control frequency of clustering");
|
||||
|
||||
// Config to control frequency of clustering
|
||||
public static final String INLINE_CLUSTERING_MAX_COMMIT_PROP = "hoodie.clustering.inline.max.commits";
|
||||
private static final String DEFAULT_INLINE_CLUSTERING_NUM_COMMITS = "4";
|
||||
|
||||
// Any strategy specific params can be saved with this prefix
|
||||
public static final String CLUSTERING_STRATEGY_PARAM_PREFIX = "hoodie.clustering.plan.strategy.";
|
||||
|
||||
// Number of partitions to list to create ClusteringPlan.
|
||||
public static final String CLUSTERING_TARGET_PARTITIONS = CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions";
|
||||
public static final String DEFAULT_CLUSTERING_TARGET_PARTITIONS = String.valueOf(2);
|
||||
public static final ConfigProperty<String> CLUSTERING_TARGET_PARTITIONS = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "daybased.lookback.partitions")
|
||||
.defaultValue("2")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Number of partitions to list to create ClusteringPlan");
|
||||
|
||||
// Files smaller than the size specified here are candidates for clustering.
|
||||
public static final String CLUSTERING_PLAN_SMALL_FILE_LIMIT = CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit";
|
||||
public static final String DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT = String.valueOf(600 * 1024 * 1024L); // 600MB
|
||||
public static final ConfigProperty<String> CLUSTERING_PLAN_SMALL_FILE_LIMIT = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "small.file.limit")
|
||||
.defaultValue(String.valueOf(600 * 1024 * 1024L))
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Files smaller than the size specified here are candidates for clustering");
|
||||
|
||||
// Each clustering operation can create multiple groups. Total amount of data processed by clustering operation
|
||||
// is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS).
|
||||
// Max amount of data to be included in one group
|
||||
public static final String CLUSTERING_MAX_BYTES_PER_GROUP = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group";
|
||||
public static final String DEFAULT_CLUSTERING_MAX_GROUP_SIZE = String.valueOf(2 * 1024 * 1024 * 1024L);
|
||||
public static final ConfigProperty<String> CLUSTERING_MAX_BYTES_PER_GROUP = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.bytes.per.group")
|
||||
.defaultValue(String.valueOf(2 * 1024 * 1024 * 1024L))
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Each clustering operation can create multiple groups. Total amount of data processed by clustering operation"
|
||||
+ " is defined by below two properties (CLUSTERING_MAX_BYTES_PER_GROUP * CLUSTERING_MAX_NUM_GROUPS)."
|
||||
+ " Max amount of data to be included in one group");
|
||||
|
||||
// Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism.
|
||||
public static final String CLUSTERING_MAX_NUM_GROUPS = CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups";
|
||||
public static final String DEFAULT_CLUSTERING_MAX_NUM_GROUPS = "30";
|
||||
public static final ConfigProperty<String> CLUSTERING_MAX_NUM_GROUPS = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "max.num.groups")
|
||||
.defaultValue("30")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism");
|
||||
|
||||
// Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups.
|
||||
public static final String CLUSTERING_TARGET_FILE_MAX_BYTES = CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes";
|
||||
public static final String DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES = String.valueOf(1 * 1024 * 1024 * 1024L); // 1GB
|
||||
|
||||
// Constants related to clustering that may be used by more than 1 strategy.
|
||||
public static final String CLUSTERING_SORT_COLUMNS_PROPERTY = HoodieClusteringConfig.CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns";
|
||||
public static final ConfigProperty<String> CLUSTERING_TARGET_FILE_MAX_BYTES = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "target.file.max.bytes")
|
||||
.defaultValue(String.valueOf(1 * 1024 * 1024 * 1024L))
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups");
|
||||
|
||||
// When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update
|
||||
public static final String CLUSTERING_UPDATES_STRATEGY_PROP = "hoodie.clustering.updates.strategy";
|
||||
public static final String DEFAULT_CLUSTERING_UPDATES_STRATEGY = "org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy";
|
||||
public static final ConfigProperty<String> CLUSTERING_SORT_COLUMNS_PROPERTY = ConfigProperty
|
||||
.key(CLUSTERING_STRATEGY_PARAM_PREFIX + "sort.columns")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Columns to sort the data by when clustering");
|
||||
|
||||
// Async clustering
|
||||
public static final String ASYNC_CLUSTERING_ENABLE_OPT_KEY = "hoodie.clustering.async.enabled";
|
||||
public static final String DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL = "false";
|
||||
public static final ConfigProperty<String> CLUSTERING_UPDATES_STRATEGY_PROP = ConfigProperty
|
||||
.key("hoodie.clustering.updates.strategy")
|
||||
.defaultValue("org.apache.hudi.client.clustering.update.strategy.SparkRejectUpdateStrategy")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("When file groups is in clustering, need to handle the update to these file groups. Default strategy just reject the update");
|
||||
|
||||
public HoodieClusteringConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> ASYNC_CLUSTERING_ENABLE_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.clustering.async.enabled")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Async clustering");
|
||||
|
||||
private HoodieClusteringConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static Builder newBuilder() {
|
||||
@@ -94,106 +118,83 @@ public class HoodieClusteringConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieClusteringConfig clusteringConfig = new HoodieClusteringConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.clusteringConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder withClusteringPlanStrategyClass(String clusteringStrategyClass) {
|
||||
props.setProperty(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
|
||||
clusteringConfig.setValue(CLUSTERING_PLAN_STRATEGY_CLASS, clusteringStrategyClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringExecutionStrategyClass(String runClusteringStrategyClass) {
|
||||
props.setProperty(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
|
||||
clusteringConfig.setValue(CLUSTERING_EXECUTION_STRATEGY_CLASS, runClusteringStrategyClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringTargetPartitions(int clusteringTargetPartitions) {
|
||||
props.setProperty(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
|
||||
clusteringConfig.setValue(CLUSTERING_TARGET_PARTITIONS, String.valueOf(clusteringTargetPartitions));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringPlanSmallFileLimit(long clusteringSmallFileLimit) {
|
||||
props.setProperty(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
|
||||
clusteringConfig.setValue(CLUSTERING_PLAN_SMALL_FILE_LIMIT, String.valueOf(clusteringSmallFileLimit));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringSortColumns(String sortColumns) {
|
||||
props.setProperty(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
|
||||
clusteringConfig.setValue(CLUSTERING_SORT_COLUMNS_PROPERTY, sortColumns);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringMaxBytesInGroup(long clusteringMaxGroupSize) {
|
||||
props.setProperty(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
|
||||
clusteringConfig.setValue(CLUSTERING_MAX_BYTES_PER_GROUP, String.valueOf(clusteringMaxGroupSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringMaxNumGroups(int maxNumGroups) {
|
||||
props.setProperty(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
|
||||
clusteringConfig.setValue(CLUSTERING_MAX_NUM_GROUPS, String.valueOf(maxNumGroups));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringTargetFileMaxBytes(long targetFileSize) {
|
||||
props.setProperty(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
|
||||
clusteringConfig.setValue(CLUSTERING_TARGET_FILE_MAX_BYTES, String.valueOf(targetFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineClustering(Boolean inlineClustering) {
|
||||
props.setProperty(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering));
|
||||
clusteringConfig.setValue(INLINE_CLUSTERING_PROP, String.valueOf(inlineClustering));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineClusteringNumCommits(int numCommits) {
|
||||
props.setProperty(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
|
||||
clusteringConfig.setValue(INLINE_CLUSTERING_MAX_COMMIT_PROP, String.valueOf(numCommits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.clusteringConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withClusteringUpdatesStrategy(String updatesStrategyClass) {
|
||||
props.setProperty(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass);
|
||||
clusteringConfig.setValue(CLUSTERING_UPDATES_STRATEGY_PROP, updatesStrategyClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncClustering(Boolean asyncClustering) {
|
||||
props.setProperty(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering));
|
||||
clusteringConfig.setValue(ASYNC_CLUSTERING_ENABLE_OPT_KEY, String.valueOf(asyncClustering));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieClusteringConfig build() {
|
||||
HoodieClusteringConfig config = new HoodieClusteringConfig(props);
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_STRATEGY_CLASS),
|
||||
CLUSTERING_PLAN_STRATEGY_CLASS, DEFAULT_CLUSTERING_PLAN_STRATEGY_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_EXECUTION_STRATEGY_CLASS),
|
||||
CLUSTERING_EXECUTION_STRATEGY_CLASS, DEFAULT_CLUSTERING_EXECUTION_STRATEGY_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_BYTES_PER_GROUP), CLUSTERING_MAX_BYTES_PER_GROUP,
|
||||
DEFAULT_CLUSTERING_MAX_GROUP_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_MAX_NUM_GROUPS), CLUSTERING_MAX_NUM_GROUPS,
|
||||
DEFAULT_CLUSTERING_MAX_NUM_GROUPS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_FILE_MAX_BYTES), CLUSTERING_TARGET_FILE_MAX_BYTES,
|
||||
DEFAULT_CLUSTERING_TARGET_FILE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_PROP), INLINE_CLUSTERING_PROP,
|
||||
DEFAULT_INLINE_CLUSTERING);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_CLUSTERING_MAX_COMMIT_PROP), INLINE_CLUSTERING_MAX_COMMIT_PROP,
|
||||
DEFAULT_INLINE_CLUSTERING_NUM_COMMITS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_TARGET_PARTITIONS), CLUSTERING_TARGET_PARTITIONS,
|
||||
DEFAULT_CLUSTERING_TARGET_PARTITIONS);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_PLAN_SMALL_FILE_LIMIT), CLUSTERING_PLAN_SMALL_FILE_LIMIT,
|
||||
DEFAULT_CLUSTERING_PLAN_SMALL_FILE_LIMIT);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLUSTERING_UPDATES_STRATEGY_PROP), CLUSTERING_UPDATES_STRATEGY_PROP,
|
||||
DEFAULT_CLUSTERING_UPDATES_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(ASYNC_CLUSTERING_ENABLE_OPT_KEY), ASYNC_CLUSTERING_ENABLE_OPT_KEY,
|
||||
DEFAULT_ASYNC_CLUSTERING_ENABLE_OPT_VAL);
|
||||
return config;
|
||||
clusteringConfig.setDefaults(HoodieClusteringConfig.class.getName());
|
||||
return clusteringConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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.model.HoodieCleaningPolicy;
|
||||
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
|
||||
import org.apache.hudi.common.model.OverwriteWithLatestAvroPayload;
|
||||
@@ -38,99 +39,176 @@ import java.util.Properties;
|
||||
* Compaction related config.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
public class HoodieCompactionConfig extends HoodieConfig {
|
||||
|
||||
public static final String CLEANER_POLICY_PROP = "hoodie.cleaner.policy";
|
||||
public static final String AUTO_CLEAN_PROP = "hoodie.clean.automatic";
|
||||
public static final String ASYNC_CLEAN_PROP = "hoodie.clean.async";
|
||||
// Turn on inline compaction - after fw delta commits a inline compaction will be run
|
||||
public static final String INLINE_COMPACT_PROP = "hoodie.compact.inline";
|
||||
// Run a compaction every N delta commits
|
||||
public static final String INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = "hoodie.compact.inline.max.delta.commits";
|
||||
// Run a compaction when time elapsed > N seconds since last compaction
|
||||
public static final String INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = "hoodie.compact.inline.max.delta.seconds";
|
||||
public static final String INLINE_COMPACT_TRIGGER_STRATEGY_PROP = "hoodie.compact.inline.trigger.strategy";
|
||||
public static final String CLEANER_FILE_VERSIONS_RETAINED_PROP = "hoodie.cleaner.fileversions.retained";
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = "hoodie.cleaner.commits.retained";
|
||||
public static final String CLEANER_INCREMENTAL_MODE = "hoodie.cleaner.incremental.mode";
|
||||
public static final String MAX_COMMITS_TO_KEEP_PROP = "hoodie.keep.max.commits";
|
||||
public static final String MIN_COMMITS_TO_KEEP_PROP = "hoodie.keep.min.commits";
|
||||
public static final String COMMITS_ARCHIVAL_BATCH_SIZE_PROP = "hoodie.commits.archival.batch";
|
||||
// Set true to clean bootstrap source files when necessary
|
||||
public static final String CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "hoodie.cleaner.delete.bootstrap.base.file";
|
||||
// Upsert uses this file size to compact new data onto existing files..
|
||||
public static final String PARQUET_SMALL_FILE_LIMIT_BYTES = "hoodie.parquet.small.file.limit";
|
||||
// By default, treat any file <= 100MB as a small file.
|
||||
public static final String DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES = String.valueOf(104857600);
|
||||
// Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten.
|
||||
// If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order,
|
||||
// until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)
|
||||
public static final String RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = "hoodie.record.size.estimation.threshold";
|
||||
public static final String DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD = "1.0";
|
||||
public static final ConfigProperty<String> CLEANER_POLICY_PROP = ConfigProperty
|
||||
.key("hoodie.cleaner.policy")
|
||||
.defaultValue(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name())
|
||||
.withDocumentation("Cleaning policy to be used. Hudi will delete older versions of parquet files to re-claim space."
|
||||
+ " Any Query/Computation referring to this version of the file will fail. "
|
||||
+ "It is good to make sure that the data is retained for more than the maximum query execution time.");
|
||||
|
||||
public static final ConfigProperty<String> AUTO_CLEAN_PROP = ConfigProperty
|
||||
.key("hoodie.clean.automatic")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Should cleanup if there is anything to cleanup immediately after the commit");
|
||||
|
||||
public static final ConfigProperty<String> ASYNC_CLEAN_PROP = ConfigProperty
|
||||
.key("hoodie.clean.async")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Only applies when #withAutoClean is turned on. When turned on runs cleaner async with writing.");
|
||||
|
||||
public static final ConfigProperty<String> INLINE_COMPACT_PROP = ConfigProperty
|
||||
.key("hoodie.compact.inline")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When set to true, compaction is triggered by the ingestion itself, "
|
||||
+ "right after a commit/deltacommit action as part of insert/upsert/bulk_insert");
|
||||
|
||||
public static final ConfigProperty<String> INLINE_COMPACT_NUM_DELTA_COMMITS_PROP = ConfigProperty
|
||||
.key("hoodie.compact.inline.max.delta.commits")
|
||||
.defaultValue("5")
|
||||
.withDocumentation("Number of max delta commits to keep before triggering an inline compaction");
|
||||
|
||||
public static final ConfigProperty<String> INLINE_COMPACT_TIME_DELTA_SECONDS_PROP = ConfigProperty
|
||||
.key("hoodie.compact.inline.max.delta.seconds")
|
||||
.defaultValue(String.valueOf(60 * 60))
|
||||
.withDocumentation("Run a compaction when time elapsed > N seconds since last compaction");
|
||||
|
||||
public static final ConfigProperty<String> INLINE_COMPACT_TRIGGER_STRATEGY_PROP = ConfigProperty
|
||||
.key("hoodie.compact.inline.trigger.strategy")
|
||||
.defaultValue(CompactionTriggerStrategy.NUM_COMMITS.name())
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> CLEANER_FILE_VERSIONS_RETAINED_PROP = ConfigProperty
|
||||
.key("hoodie.cleaner.fileversions.retained")
|
||||
.defaultValue("3")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
|
||||
.key("hoodie.cleaner.commits.retained")
|
||||
.defaultValue("10")
|
||||
.withDocumentation("Number of commits to retain. So data will be retained for num_of_commits * time_between_commits "
|
||||
+ "(scheduled). This also directly translates into how much you can incrementally pull on this table");
|
||||
|
||||
public static final ConfigProperty<String> CLEANER_INCREMENTAL_MODE = ConfigProperty
|
||||
.key("hoodie.cleaner.incremental.mode")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
.key("hoodie.keep.max.commits")
|
||||
.defaultValue("30")
|
||||
.withDocumentation("Each commit is a small file in the .hoodie directory. Since DFS typically does not favor lots of "
|
||||
+ "small files, Hudi archives older commits into a sequential log. A commit is published atomically "
|
||||
+ "by a rename of the commit file.");
|
||||
|
||||
public static final ConfigProperty<String> MIN_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
.key("hoodie.keep.min.commits")
|
||||
.defaultValue("20")
|
||||
.withDocumentation("Each commit is a small file in the .hoodie directory. Since DFS typically does not favor lots of "
|
||||
+ "small files, Hudi archives older commits into a sequential log. A commit is published atomically "
|
||||
+ "by a rename of the commit file.");
|
||||
|
||||
public static final ConfigProperty<String> COMMITS_ARCHIVAL_BATCH_SIZE_PROP = ConfigProperty
|
||||
.key("hoodie.commits.archival.batch")
|
||||
.defaultValue(String.valueOf(10))
|
||||
.withDocumentation("This controls the number of commit instants read in memory as a batch and archived together.");
|
||||
|
||||
public static final ConfigProperty<String> CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = ConfigProperty
|
||||
.key("hoodie.cleaner.delete.bootstrap.base.file")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Set true to clean bootstrap source files when necessary");
|
||||
|
||||
public static final ConfigProperty<String> PARQUET_SMALL_FILE_LIMIT_BYTES = ConfigProperty
|
||||
.key("hoodie.parquet.small.file.limit")
|
||||
.defaultValue(String.valueOf(104857600))
|
||||
.withDocumentation("Upsert uses this file size to compact new data onto existing files. "
|
||||
+ "By default, treat any file <= 100MB as a small file.");
|
||||
|
||||
public static final ConfigProperty<String> RECORD_SIZE_ESTIMATION_THRESHOLD_PROP = ConfigProperty
|
||||
.key("hoodie.record.size.estimation.threshold")
|
||||
.defaultValue("1.0")
|
||||
.withDocumentation("Hudi will use the previous commit to calculate the estimated record size by totalBytesWritten/totalRecordsWritten. "
|
||||
+ "If the previous commit is too small to make an accurate estimation, Hudi will search commits in the reverse order, "
|
||||
+ "until find a commit has totalBytesWritten larger than (PARQUET_SMALL_FILE_LIMIT_BYTES * RECORD_SIZE_ESTIMATION_THRESHOLD)");
|
||||
|
||||
public static final ConfigProperty<String> CLEANER_PARALLELISM = ConfigProperty
|
||||
.key("hoodie.cleaner.parallelism")
|
||||
.defaultValue("200")
|
||||
.withDocumentation("Increase this if cleaning becomes slow.");
|
||||
|
||||
// 500GB of target IO per compaction (both read and write
|
||||
public static final ConfigProperty<String> TARGET_IO_PER_COMPACTION_IN_MB_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.target.io")
|
||||
.defaultValue(String.valueOf(500 * 1024))
|
||||
.withDocumentation("Amount of MBs to spend during compaction run for the LogFileSizeBasedCompactionStrategy. "
|
||||
+ "This value helps bound ingestion latency while compaction is run inline mode.");
|
||||
|
||||
public static final ConfigProperty<String> COMPACTION_STRATEGY_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.strategy")
|
||||
.defaultValue(LogFileSizeBasedCompactionStrategy.class.getName())
|
||||
.withDocumentation("Compaction strategy decides which file groups are picked up for "
|
||||
+ "compaction during each compaction run. By default. Hudi picks the log file "
|
||||
+ "with most accumulated unmerged data");
|
||||
|
||||
public static final ConfigProperty<String> PAYLOAD_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.payload.class")
|
||||
.defaultValue(OverwriteWithLatestAvroPayload.class.getName())
|
||||
.withDocumentation("This needs to be same as class used during insert/upserts. Just like writing, compaction also uses "
|
||||
+ "the record payload class to merge records in the log against each other, merge again with the base file and "
|
||||
+ "produce the final record to be written after compaction.");
|
||||
|
||||
public static final ConfigProperty<String> COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.lazy.block.read")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("When a CompactedLogScanner merges all log files, this config helps to choose whether the logblocks "
|
||||
+ "should be read lazily or not. Choose true to use I/O intensive lazy block reading (low memory usage) or false "
|
||||
+ "for Memory intensive immediate block read (high memory usage)");
|
||||
|
||||
public static final ConfigProperty<String> COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.reverse.log.read")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("HoodieLogFormatReader reads a logfile in the forward direction starting from pos=0 to pos=file_length. "
|
||||
+ "If this config is set to true, the Reader reads the logfile in reverse direction, from pos=file_length to pos=0");
|
||||
|
||||
public static final ConfigProperty<String> FAILED_WRITES_CLEANER_POLICY_PROP = ConfigProperty
|
||||
.key("hoodie.cleaner.policy.failed.writes")
|
||||
.defaultValue(HoodieFailedWritesCleaningPolicy.EAGER.name())
|
||||
.withDocumentation("Cleaning policy for failed writes to be used. Hudi will delete any files written by "
|
||||
+ "failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before "
|
||||
+ "every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)");
|
||||
|
||||
public static final ConfigProperty<String> TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.daybased.target.partitions")
|
||||
.defaultValue("10")
|
||||
.withDocumentation("Used by org.apache.hudi.io.compact.strategy.DayBasedCompactionStrategy to denote the number of "
|
||||
+ "latest partitions to compact during a compaction run.");
|
||||
|
||||
/**
|
||||
* Configs related to specific table types.
|
||||
*/
|
||||
// Number of inserts, that will be put each partition/bucket for writing
|
||||
public static final String COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = "hoodie.copyonwrite.insert.split.size";
|
||||
// The rationale to pick the insert parallelism is the following. Writing out 100MB files,
|
||||
// with atleast 1kb records, means 100K records per file. we just overprovision to 500K
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = String.valueOf(500000);
|
||||
// Config to control whether we control insert split sizes automatically based on average
|
||||
// record sizes
|
||||
public static final String COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = "hoodie.copyonwrite.insert.auto.split";
|
||||
// its off by default
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = String.valueOf(true);
|
||||
// This value is used as a guesstimate for the record size, if we can't determine this from
|
||||
// previous commits
|
||||
public static final String COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = "hoodie.copyonwrite.record.size.estimate";
|
||||
// Used to determine how much more can be packed into a small file, before it exceeds the size
|
||||
// limit.
|
||||
public static final String DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = String.valueOf(1024);
|
||||
public static final String CLEANER_PARALLELISM = "hoodie.cleaner.parallelism";
|
||||
public static final String DEFAULT_CLEANER_PARALLELISM = String.valueOf(200);
|
||||
public static final String TARGET_IO_PER_COMPACTION_IN_MB_PROP = "hoodie.compaction.target.io";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB = String.valueOf(500 * 1024);
|
||||
public static final String COMPACTION_STRATEGY_PROP = "hoodie.compaction.strategy";
|
||||
// 200GB of target IO per compaction
|
||||
public static final String DEFAULT_COMPACTION_STRATEGY = LogFileSizeBasedCompactionStrategy.class.getName();
|
||||
// used to merge records written to log file
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = OverwriteWithLatestAvroPayload.class.getName();
|
||||
public static final String PAYLOAD_CLASS_PROP = "hoodie.compaction.payload.class";
|
||||
public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE = ConfigProperty
|
||||
.key("hoodie.copyonwrite.insert.split.size")
|
||||
.defaultValue(String.valueOf(500000))
|
||||
.withDocumentation("Number of inserts, that will be put each partition/bucket for writing. "
|
||||
+ "The rationale to pick the insert parallelism is the following. Writing out 100MB files, "
|
||||
+ "with at least 1kb records, means 100K records per file. we just over provision to 500K.");
|
||||
|
||||
// used to choose a trade off between IO vs Memory when performing compaction process
|
||||
// Depending on outputfile_size and memory provided, choose true to avoid OOM for large file
|
||||
// size + small memory
|
||||
public static final String COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP = "hoodie.compaction.lazy.block.read";
|
||||
public static final String DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED = "false";
|
||||
// used to choose whether to enable reverse log reading (reverse log traversal)
|
||||
public static final String COMPACTION_REVERSE_LOG_READ_ENABLED_PROP = "hoodie.compaction.reverse.log.read";
|
||||
public static final String DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED = "false";
|
||||
private static final String DEFAULT_CLEANER_POLICY = HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name();
|
||||
public static final String FAILED_WRITES_CLEANER_POLICY_PROP = "hoodie.cleaner.policy.failed.writes";
|
||||
private static final String DEFAULT_FAILED_WRITES_CLEANER_POLICY =
|
||||
HoodieFailedWritesCleaningPolicy.EAGER.name();
|
||||
private static final String DEFAULT_AUTO_CLEAN = "true";
|
||||
private static final String DEFAULT_ASYNC_CLEAN = "false";
|
||||
private static final String DEFAULT_INLINE_COMPACT = "false";
|
||||
private static final String DEFAULT_INCREMENTAL_CLEANER = "true";
|
||||
private static final String DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS = "5";
|
||||
private static final String DEFAULT_INLINE_COMPACT_TIME_DELTA_SECONDS = String.valueOf(60 * 60);
|
||||
private static final String DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY = CompactionTriggerStrategy.NUM_COMMITS.name();
|
||||
private static final String DEFAULT_CLEANER_FILE_VERSIONS_RETAINED = "3";
|
||||
private static final String DEFAULT_CLEANER_COMMITS_RETAINED = "10";
|
||||
private static final String DEFAULT_MAX_COMMITS_TO_KEEP = "30";
|
||||
private static final String DEFAULT_MIN_COMMITS_TO_KEEP = "20";
|
||||
private static final String DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE = String.valueOf(10);
|
||||
private static final String DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED = "false";
|
||||
public static final String TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP =
|
||||
"hoodie.compaction.daybased.target.partitions";
|
||||
// 500GB of target IO per compaction (both read and write)
|
||||
public static final String DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION = String.valueOf(10);
|
||||
public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS = ConfigProperty
|
||||
.key("hoodie.copyonwrite.insert.auto.split")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Config to control whether we control insert split sizes automatically based on average"
|
||||
+ " record sizes.");
|
||||
|
||||
private HoodieCompactionConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE = ConfigProperty
|
||||
.key("hoodie.copyonwrite.record.size.estimate")
|
||||
.defaultValue(String.valueOf(1024))
|
||||
.withDocumentation("The average record size. If specified, hudi will use this and not compute dynamically "
|
||||
+ "based on the last 24 commit’s metadata. No value set as default. This is critical in computing "
|
||||
+ "the insert parallelism and bin-packing inserts into small files. See above.");
|
||||
|
||||
private HoodieCompactionConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieCompactionConfig.Builder newBuilder() {
|
||||
@@ -139,226 +217,174 @@ public class HoodieCompactionConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieCompactionConfig compactionConfig = new HoodieCompactionConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.compactionConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.compactionConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAutoClean(Boolean autoClean) {
|
||||
props.setProperty(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
||||
compactionConfig.setValue(AUTO_CLEAN_PROP, String.valueOf(autoClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncClean(Boolean asyncClean) {
|
||||
props.setProperty(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
compactionConfig.setValue(ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalCleaningMode(Boolean incrementalCleaningMode) {
|
||||
props.setProperty(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
|
||||
compactionConfig.setValue(CLEANER_INCREMENTAL_MODE, String.valueOf(incrementalCleaningMode));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompaction(Boolean inlineCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||
compactionConfig.setValue(INLINE_COMPACT_PROP, String.valueOf(inlineCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInlineCompactionTriggerStrategy(CompactionTriggerStrategy compactionTriggerStrategy) {
|
||||
props.setProperty(INLINE_COMPACT_TRIGGER_STRATEGY_PROP, compactionTriggerStrategy.name());
|
||||
compactionConfig.setValue(INLINE_COMPACT_TRIGGER_STRATEGY_PROP, compactionTriggerStrategy.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerPolicy(HoodieCleaningPolicy policy) {
|
||||
props.setProperty(CLEANER_POLICY_PROP, policy.name());
|
||||
compactionConfig.setValue(CLEANER_POLICY_PROP, policy.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainFileVersions(int fileVersionsRetained) {
|
||||
props.setProperty(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
|
||||
compactionConfig.setValue(CLEANER_FILE_VERSIONS_RETAINED_PROP, String.valueOf(fileVersionsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainCommits(int commitsRetained) {
|
||||
props.setProperty(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
compactionConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
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));
|
||||
compactionConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
compactionConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionSmallFileSize(long smallFileLimitBytes) {
|
||||
props.setProperty(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
||||
compactionConfig.setValue(PARQUET_SMALL_FILE_LIMIT_BYTES, String.valueOf(smallFileLimitBytes));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder compactionRecordSizeEstimateThreshold(double threshold) {
|
||||
props.setProperty(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
|
||||
compactionConfig.setValue(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP, String.valueOf(threshold));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder insertSplitSize(int insertSplitSize) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||
compactionConfig.setValue(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, String.valueOf(insertSplitSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder autoTuneInsertSplits(boolean autoTuneInsertSplits) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
|
||||
compactionConfig.setValue(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, String.valueOf(autoTuneInsertSplits));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder approxRecordSize(int recordSizeEstimate) {
|
||||
props.setProperty(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
|
||||
compactionConfig.setValue(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, String.valueOf(recordSizeEstimate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanerParallelism(int cleanerParallelism) {
|
||||
props.setProperty(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
||||
compactionConfig.setValue(CLEANER_PARALLELISM, String.valueOf(cleanerParallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionStrategy(CompactionStrategy compactionStrategy) {
|
||||
props.setProperty(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
||||
compactionConfig.setValue(COMPACTION_STRATEGY_PROP, compactionStrategy.getClass().getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPayloadClass(String payloadClassName) {
|
||||
props.setProperty(PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
compactionConfig.setValue(PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetIOPerCompactionInMB(long targetIOPerCompactionInMB) {
|
||||
props.setProperty(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
|
||||
compactionConfig.setValue(TARGET_IO_PER_COMPACTION_IN_MB_PROP, String.valueOf(targetIOPerCompactionInMB));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
compactionConfig.setValue(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxDeltaSecondsBeforeCompaction(int maxDeltaSecondsBeforeCompaction) {
|
||||
props.setProperty(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, String.valueOf(maxDeltaSecondsBeforeCompaction));
|
||||
compactionConfig.setValue(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, String.valueOf(maxDeltaSecondsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionLazyBlockReadEnabled(Boolean compactionLazyBlockReadEnabled) {
|
||||
props.setProperty(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, String.valueOf(compactionLazyBlockReadEnabled));
|
||||
compactionConfig.setValue(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, String.valueOf(compactionLazyBlockReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCompactionReverseLogReadEnabled(Boolean compactionReverseLogReadEnabled) {
|
||||
props.setProperty(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, String.valueOf(compactionReverseLogReadEnabled));
|
||||
compactionConfig.setValue(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, String.valueOf(compactionReverseLogReadEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withTargetPartitionsPerDayBasedCompaction(int targetPartitionsPerCompaction) {
|
||||
props.setProperty(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, String.valueOf(targetPartitionsPerCompaction));
|
||||
compactionConfig.setValue(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, String.valueOf(targetPartitionsPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCommitsArchivalBatchSize(int batchSize) {
|
||||
props.setProperty(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
|
||||
compactionConfig.setValue(COMMITS_ARCHIVAL_BATCH_SIZE_PROP, String.valueOf(batchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCleanBootstrapBaseFileEnabled(Boolean cleanBootstrapSourceFileEnabled) {
|
||||
props.setProperty(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
|
||||
compactionConfig.setValue(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, String.valueOf(cleanBootstrapSourceFileEnabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy failedWritesPolicy) {
|
||||
props.setProperty(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name());
|
||||
compactionConfig.setValue(FAILED_WRITES_CLEANER_POLICY_PROP, failedWritesPolicy.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieCompactionConfig build() {
|
||||
HoodieCompactionConfig config = new HoodieCompactionConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(AUTO_CLEAN_PROP), AUTO_CLEAN_PROP, DEFAULT_AUTO_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(ASYNC_CLEAN_PROP), ASYNC_CLEAN_PROP,
|
||||
DEFAULT_ASYNC_CLEAN);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_INCREMENTAL_MODE), CLEANER_INCREMENTAL_MODE,
|
||||
DEFAULT_INCREMENTAL_CLEANER);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_PROP), INLINE_COMPACT_PROP,
|
||||
DEFAULT_INLINE_COMPACT);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_NUM_DELTA_COMMITS_PROP),
|
||||
INLINE_COMPACT_NUM_DELTA_COMMITS_PROP, DEFAULT_INLINE_COMPACT_NUM_DELTA_COMMITS);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_TIME_DELTA_SECONDS_PROP),
|
||||
INLINE_COMPACT_TIME_DELTA_SECONDS_PROP, DEFAULT_INLINE_COMPACT_TIME_DELTA_SECONDS);
|
||||
setDefaultOnCondition(props, !props.containsKey(INLINE_COMPACT_TRIGGER_STRATEGY_PROP),
|
||||
INLINE_COMPACT_TRIGGER_STRATEGY_PROP, DEFAULT_INLINE_COMPACT_TRIGGER_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_POLICY_PROP), CLEANER_POLICY_PROP,
|
||||
DEFAULT_CLEANER_POLICY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_FILE_VERSIONS_RETAINED_PROP),
|
||||
CLEANER_FILE_VERSIONS_RETAINED_PROP, DEFAULT_CLEANER_FILE_VERSIONS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_COMMITS_RETAINED_PROP), CLEANER_COMMITS_RETAINED_PROP,
|
||||
DEFAULT_CLEANER_COMMITS_RETAINED);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_COMMITS_TO_KEEP_PROP), MAX_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MAX_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(MIN_COMMITS_TO_KEEP_PROP), MIN_COMMITS_TO_KEEP_PROP,
|
||||
DEFAULT_MIN_COMMITS_TO_KEEP);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_SMALL_FILE_LIMIT_BYTES), PARQUET_SMALL_FILE_LIMIT_BYTES,
|
||||
DEFAULT_PARQUET_SMALL_FILE_LIMIT_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(RECORD_SIZE_ESTIMATION_THRESHOLD_PROP), RECORD_SIZE_ESTIMATION_THRESHOLD_PROP,
|
||||
DEFAULT_RECORD_SIZE_ESTIMATION_THRESHOLD);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE),
|
||||
COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE, DEFAULT_COPY_ON_WRITE_TABLE_INSERT_SPLIT_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS),
|
||||
COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS, DEFAULT_COPY_ON_WRITE_TABLE_AUTO_SPLIT_INSERTS);
|
||||
setDefaultOnCondition(props, !props.containsKey(COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE),
|
||||
COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE, DEFAULT_COPY_ON_WRITE_TABLE_RECORD_SIZE_ESTIMATE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_PARALLELISM), CLEANER_PARALLELISM,
|
||||
DEFAULT_CLEANER_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_STRATEGY_PROP), COMPACTION_STRATEGY_PROP,
|
||||
DEFAULT_COMPACTION_STRATEGY);
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_CLASS_PROP), PAYLOAD_CLASS_PROP, DEFAULT_PAYLOAD_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_IO_PER_COMPACTION_IN_MB_PROP),
|
||||
TARGET_IO_PER_COMPACTION_IN_MB_PROP, DEFAULT_TARGET_IO_PER_COMPACTION_IN_MB);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP),
|
||||
COMPACTION_LAZY_BLOCK_READ_ENABLED_PROP, DEFAULT_COMPACTION_LAZY_BLOCK_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMPACTION_REVERSE_LOG_READ_ENABLED_PROP),
|
||||
COMPACTION_REVERSE_LOG_READ_ENABLED_PROP, DEFAULT_COMPACTION_REVERSE_LOG_READ_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP),
|
||||
TARGET_PARTITIONS_PER_DAYBASED_COMPACTION_PROP, DEFAULT_TARGET_PARTITIONS_PER_DAYBASED_COMPACTION);
|
||||
setDefaultOnCondition(props, !props.containsKey(COMMITS_ARCHIVAL_BATCH_SIZE_PROP),
|
||||
COMMITS_ARCHIVAL_BATCH_SIZE_PROP, DEFAULT_COMMITS_ARCHIVAL_BATCH_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(CLEANER_BOOTSTRAP_BASE_FILE_ENABLED),
|
||||
CLEANER_BOOTSTRAP_BASE_FILE_ENABLED, DEFAULT_CLEANER_BOOTSTRAP_BASE_FILE_ENABLED);
|
||||
setDefaultOnCondition(props, !props.containsKey(FAILED_WRITES_CLEANER_POLICY_PROP),
|
||||
FAILED_WRITES_CLEANER_POLICY_PROP, DEFAULT_FAILED_WRITES_CLEANER_POLICY);
|
||||
|
||||
HoodieCleaningPolicy.valueOf(props.getProperty(CLEANER_POLICY_PROP));
|
||||
compactionConfig.setDefaults(HoodieCompactionConfig.class.getName());
|
||||
// validation
|
||||
HoodieCleaningPolicy.valueOf(compactionConfig.getString(CLEANER_POLICY_PROP));
|
||||
|
||||
// Ensure minInstantsToKeep > cleanerCommitsRetained, otherwise we will archive some
|
||||
// commit instant on timeline, that still has not been cleaned. Could miss some data via incr pull
|
||||
int minInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
int maxInstantsToKeep = Integer.parseInt(props.getProperty(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
int minInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP));
|
||||
int maxInstantsToKeep = Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP));
|
||||
int cleanerCommitsRetained =
|
||||
Integer.parseInt(props.getProperty(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
Integer.parseInt(compactionConfig.getStringOrDefault(HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP));
|
||||
ValidationUtils.checkArgument(maxInstantsToKeep > minInstantsToKeep,
|
||||
String.format(
|
||||
"Increase %s=%d to be greater than %s=%d.",
|
||||
HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP, maxInstantsToKeep,
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep));
|
||||
HoodieCompactionConfig.MAX_COMMITS_TO_KEEP_PROP.key(), maxInstantsToKeep,
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), minInstantsToKeep));
|
||||
ValidationUtils.checkArgument(minInstantsToKeep > cleanerCommitsRetained,
|
||||
String.format(
|
||||
"Increase %s=%d to be greater than %s=%d. Otherwise, there is risk of incremental pull "
|
||||
+ "missing data from few instants.",
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP, minInstantsToKeep,
|
||||
HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP, cleanerCommitsRetained));
|
||||
return config;
|
||||
HoodieCompactionConfig.MIN_COMMITS_TO_KEEP_PROP.key(), minInstantsToKeep,
|
||||
HoodieCompactionConfig.CLEANER_COMMITS_RETAINED_PROP.key(), cleanerCommitsRetained));
|
||||
return compactionConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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.index.hbase.DefaultHBaseQPSResourceAllocator;
|
||||
|
||||
import java.io.File;
|
||||
@@ -26,102 +27,127 @@ import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
|
||||
public class HoodieHBaseIndexConfig extends HoodieConfig {
|
||||
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
/**
|
||||
* Note that if HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP is set to true, this batch size will not be honored for HBase
|
||||
* Puts.
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public static final ConfigProperty<String> HBASE_ZKQUORUM_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.zkquorum")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum url to connect to");
|
||||
|
||||
/**
|
||||
* Property to set which implementation of HBase QPS resource allocator to be used.
|
||||
*/
|
||||
public static final String HBASE_INDEX_QPS_ALLOCATOR_CLASS = "hoodie.index.hbase.qps.allocator.class";
|
||||
public static final String DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS = DefaultHBaseQPSResourceAllocator.class.getName();
|
||||
/**
|
||||
* Property to set to enable auto computation of put batch size.
|
||||
*/
|
||||
public static final String HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = "hoodie.index.hbase.put.batch.size.autocompute";
|
||||
public static final String DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE = "false";
|
||||
/**
|
||||
* Property to set the fraction of the global share of QPS that should be allocated to this job. Let's say there are 3
|
||||
* jobs which have input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then
|
||||
* this fraction for the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively.
|
||||
*/
|
||||
public static final String HBASE_QPS_FRACTION_PROP = "hoodie.index.hbase.qps.fraction";
|
||||
/**
|
||||
* Property to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to
|
||||
* limit the aggregate QPS generated across various jobs to an Hbase Region Server. It is recommended to set this
|
||||
* value based on global indexing throughput needs and most importantly, how much the HBase installation in use is
|
||||
* able to tolerate without Region Servers going down.
|
||||
*/
|
||||
public static final String HBASE_MAX_QPS_PER_REGION_SERVER_PROP = "hoodie.index.hbase.max.qps.per.region.server";
|
||||
/**
|
||||
* Default batch size, used only for Get, but computed for Put.
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_BATCH_SIZE = 100;
|
||||
/**
|
||||
* A low default value.
|
||||
*/
|
||||
public static final int DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER = 1000;
|
||||
/**
|
||||
* Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming Region Servers.
|
||||
*/
|
||||
public static final float DEFAULT_HBASE_QPS_FRACTION = 0.5f;
|
||||
public static final ConfigProperty<String> HBASE_ZKPORT_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.zkport")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Only applies if index type is HBASE. HBase ZK Quorum port to connect to");
|
||||
|
||||
/**
|
||||
* Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume.
|
||||
*/
|
||||
public static final String HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = "hoodie.index.hbase.dynamic_qps";
|
||||
public static final boolean DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = false;
|
||||
/**
|
||||
* Min and Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads.
|
||||
*/
|
||||
public static final String HBASE_MIN_QPS_FRACTION_PROP = "hoodie.index.hbase.min.qps.fraction";
|
||||
public static final ConfigProperty<String> HBASE_TABLENAME_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.table")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Only applies if index type is HBASE. HBase Table name to use as the index. "
|
||||
+ "Hudi stores the row_key and [partition_path, fileID, commitTime] mapping in the table");
|
||||
|
||||
public static final String HBASE_MAX_QPS_FRACTION_PROP = "hoodie.index.hbase.max.qps.fraction";
|
||||
public static final ConfigProperty<Integer> HBASE_GET_BATCH_SIZE_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.get.batch.size")
|
||||
.defaultValue(100)
|
||||
.withDocumentation("");
|
||||
|
||||
/**
|
||||
* Hoodie index desired puts operation time in seconds.
|
||||
*/
|
||||
public static final String HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = "hoodie.index.hbase.desired_puts_time_in_secs";
|
||||
public static final int DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = 600;
|
||||
public static final String HBASE_SLEEP_MS_PUT_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.put.batch";
|
||||
public static final String HBASE_SLEEP_MS_GET_BATCH_PROP = "hoodie.index.hbase.sleep.ms.for.get.batch";
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = "hoodie.index.hbase.zk.session_timeout_ms";
|
||||
public static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 60 * 1000;
|
||||
public static final String HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS =
|
||||
"hoodie.index.hbase.zk.connection_timeout_ms";
|
||||
public static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 15 * 1000;
|
||||
public static final String HBASE_ZK_PATH_QPS_ROOT = "hoodie.index.hbase.zkpath.qps_root";
|
||||
public static final String DEFAULT_HBASE_ZK_PATH_QPS_ROOT = "/QPS_ROOT";
|
||||
public static final ConfigProperty<String> HBASE_ZK_ZNODEPARENT = ConfigProperty
|
||||
.key("hoodie.index.hbase.zknode.path")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Only applies if index type is HBASE. This is the root znode that will contain "
|
||||
+ "all the znodes created/used by HBase");
|
||||
|
||||
/**
|
||||
* Only applies if index type is Hbase.
|
||||
* <p>
|
||||
* When set to true, an update to a record with a different partition from its existing one
|
||||
* will insert the record to the new partition and delete it from the old partition.
|
||||
* <p>
|
||||
* When set to false, a record will be updated to the old partition.
|
||||
*/
|
||||
public static final String HBASE_INDEX_UPDATE_PARTITION_PATH = "hoodie.hbase.index.update.partition.path";
|
||||
public static final Boolean DEFAULT_HBASE_INDEX_UPDATE_PARTITION_PATH = false;
|
||||
public static final ConfigProperty<Integer> HBASE_PUT_BATCH_SIZE_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.put.batch.size")
|
||||
.defaultValue(100)
|
||||
.withDocumentation("");
|
||||
|
||||
/**
|
||||
* When set to true, the rollback method will delete the last failed task index .
|
||||
* The default value is false. Because deleting the index will add extra load on the Hbase cluster for each rollback.
|
||||
*/
|
||||
public static final String HBASE_INDEX_ROLLBACK_SYNC = "hoodie.index.hbase.rollback.sync";
|
||||
public static final Boolean DEFAULT_HBASE_INDEX_ROLLBACK_SYNC = false;
|
||||
public static final ConfigProperty<String> HBASE_INDEX_QPS_ALLOCATOR_CLASS = ConfigProperty
|
||||
.key("hoodie.index.hbase.qps.allocator.class")
|
||||
.defaultValue(DefaultHBaseQPSResourceAllocator.class.getName())
|
||||
.withDocumentation("Property to set which implementation of HBase QPS resource allocator to be used");
|
||||
|
||||
public HoodieHBaseIndexConfig(final Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.put.batch.size.autocompute")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Property to set to enable auto computation of put batch size");
|
||||
|
||||
public static final ConfigProperty<Float> HBASE_QPS_FRACTION_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.qps.fraction")
|
||||
.defaultValue(0.5f)
|
||||
.withDocumentation("Property to set the fraction of the global share of QPS that should be allocated to this job. Let's say there are 3"
|
||||
+ " jobs which have input size in terms of number of rows required for HbaseIndexing as x, 2x, 3x respectively. Then"
|
||||
+ " this fraction for the jobs would be (0.17) 1/6, 0.33 (2/6) and 0.5 (3/6) respectively."
|
||||
+ " Default is 50%, which means a total of 2 jobs can run using HbaseIndex without overwhelming Region Servers.");
|
||||
|
||||
public static final ConfigProperty<Integer> HBASE_MAX_QPS_PER_REGION_SERVER_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.max.qps.per.region.server")
|
||||
.defaultValue(1000)
|
||||
.withDocumentation("Property to set maximum QPS allowed per Region Server. This should be same across various jobs. This is intended to\n"
|
||||
+ " limit the aggregate QPS generated across various jobs to an Hbase Region Server. It is recommended to set this\n"
|
||||
+ " value based on global indexing throughput needs and most importantly, how much the HBase installation in use is\n"
|
||||
+ " able to tolerate without Region Servers going down.");
|
||||
|
||||
public static final ConfigProperty<Boolean> HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY = ConfigProperty
|
||||
.key("hoodie.index.hbase.dynamic_qps")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Property to decide if HBASE_QPS_FRACTION_PROP is dynamically calculated based on volume");
|
||||
|
||||
public static final ConfigProperty<String> HBASE_MIN_QPS_FRACTION_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.min.qps.fraction")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Min for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads");
|
||||
|
||||
public static final ConfigProperty<String> HBASE_MAX_QPS_FRACTION_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.max.qps.fraction")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Max for HBASE_QPS_FRACTION_PROP to stabilize skewed volume workloads");
|
||||
|
||||
public static final ConfigProperty<Integer> HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS = ConfigProperty
|
||||
.key("hoodie.index.hbase.desired_puts_time_in_secs")
|
||||
.defaultValue(600)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HBASE_SLEEP_MS_PUT_BATCH_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.sleep.ms.for.put.batch")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HBASE_SLEEP_MS_GET_BATCH_PROP = ConfigProperty
|
||||
.key("hoodie.index.hbase.sleep.ms.for.get.batch")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS = ConfigProperty
|
||||
.key("hoodie.index.hbase.zk.session_timeout_ms")
|
||||
.defaultValue(60 * 1000)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<Integer> HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS = ConfigProperty
|
||||
.key("hoodie.index.hbase.zk.connection_timeout_ms")
|
||||
.defaultValue(15 * 1000)
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HBASE_ZK_PATH_QPS_ROOT = ConfigProperty
|
||||
.key("hoodie.index.hbase.zkpath.qps_root")
|
||||
.defaultValue("/QPS_ROOT")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<Boolean> HBASE_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
|
||||
.key("hoodie.hbase.index.update.partition.path")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("Only applies if index type is HBASE. "
|
||||
+ "When an already existing record is upserted to a new partition compared to whats in storage, "
|
||||
+ "this config when set, will delete old record in old paritition "
|
||||
+ "and will insert it as new record in new partition.");
|
||||
|
||||
public static final ConfigProperty<Boolean> HBASE_INDEX_ROLLBACK_SYNC = ConfigProperty
|
||||
.key("hoodie.index.hbase.rollback.sync")
|
||||
.defaultValue(false)
|
||||
.withDocumentation("When set to true, the rollback method will delete the last failed task index. "
|
||||
+ "The default value is false. Because deleting the index will add extra load on the Hbase cluster for each rollback");
|
||||
|
||||
private HoodieHBaseIndexConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieHBaseIndexConfig.Builder newBuilder() {
|
||||
@@ -130,117 +156,117 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieHBaseIndexConfig hBaseIndexConfig = new HoodieHBaseIndexConfig();
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.hBaseIndexConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.hBaseIndexConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
hBaseIndexConfig.setValue(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
hBaseIndexConfig.setValue(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig.Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
hBaseIndexConfig.setValue(HBASE_TABLENAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeQPSPath(String zkZnodeQPSPath) {
|
||||
props.setProperty(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
|
||||
hBaseIndexConfig.setValue(HBASE_ZK_PATH_QPS_ROOT, zkZnodeQPSPath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexGetBatchSize(int getBatchSize) {
|
||||
props.setProperty(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
|
||||
hBaseIndexConfig.setValue(HBASE_GET_BATCH_SIZE_PROP, String.valueOf(getBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSize(int putBatchSize) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
|
||||
hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE_PROP, String.valueOf(putBatchSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexPutBatchSizeAutoCompute(boolean putBatchSizeAutoCompute) {
|
||||
props.setProperty(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
|
||||
hBaseIndexConfig.setValue(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, String.valueOf(putBatchSizeAutoCompute));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexDesiredPutsTime(int desiredPutsTime) {
|
||||
props.setProperty(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
|
||||
hBaseIndexConfig.setValue(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(desiredPutsTime));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexShouldComputeQPSDynamically(boolean shouldComputeQPsDynamically) {
|
||||
props.setProperty(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
|
||||
hBaseIndexConfig.setValue(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(shouldComputeQPsDynamically));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexQPSFraction(float qpsFraction) {
|
||||
props.setProperty(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
|
||||
hBaseIndexConfig.setValue(HBASE_QPS_FRACTION_PROP, String.valueOf(qpsFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMinQPSFraction(float minQPSFraction) {
|
||||
props.setProperty(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
|
||||
hBaseIndexConfig.setValue(HBASE_MIN_QPS_FRACTION_PROP, String.valueOf(minQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexMaxQPSFraction(float maxQPSFraction) {
|
||||
props.setProperty(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
|
||||
hBaseIndexConfig.setValue(HBASE_MAX_QPS_FRACTION_PROP, String.valueOf(maxQPSFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexSleepMsBetweenPutBatch(int sleepMsBetweenPutBatch) {
|
||||
props.setProperty(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
|
||||
hBaseIndexConfig.setValue(HBASE_SLEEP_MS_PUT_BATCH_PROP, String.valueOf(sleepMsBetweenPutBatch));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexSleepMsBetweenGetBatch(int sleepMsBetweenGetBatch) {
|
||||
props.setProperty(HBASE_SLEEP_MS_GET_BATCH_PROP, String.valueOf(sleepMsBetweenGetBatch));
|
||||
hBaseIndexConfig.setValue(HBASE_SLEEP_MS_GET_BATCH_PROP, String.valueOf(sleepMsBetweenGetBatch));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(HBASE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
hBaseIndexConfig.setValue(HBASE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexRollbackSync(boolean rollbackSync) {
|
||||
props.setProperty(HBASE_INDEX_ROLLBACK_SYNC, String.valueOf(rollbackSync));
|
||||
hBaseIndexConfig.setValue(HBASE_INDEX_ROLLBACK_SYNC, String.valueOf(rollbackSync));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withQPSResourceAllocatorType(String qpsResourceAllocatorClass) {
|
||||
props.setProperty(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
|
||||
hBaseIndexConfig.setValue(HBASE_INDEX_QPS_ALLOCATOR_CLASS, qpsResourceAllocatorClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkSessionTimeout(int zkSessionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
|
||||
hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(zkSessionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseIndexZkConnectionTimeout(int zkConnectionTimeout) {
|
||||
props.setProperty(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
|
||||
hBaseIndexConfig.setValue(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(zkConnectionTimeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
hBaseIndexConfig.setValue(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -256,42 +282,14 @@ public class HoodieHBaseIndexConfig extends DefaultHoodieConfig {
|
||||
*/
|
||||
public HoodieHBaseIndexConfig.Builder hbaseIndexMaxQPSPerRegionServer(int maxQPSPerRegionServer) {
|
||||
// This should be same across various jobs
|
||||
props.setProperty(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
|
||||
hBaseIndexConfig.setValue(HoodieHBaseIndexConfig.HBASE_MAX_QPS_PER_REGION_SERVER_PROP,
|
||||
String.valueOf(maxQPSPerRegionServer));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieHBaseIndexConfig build() {
|
||||
HoodieHBaseIndexConfig config = new HoodieHBaseIndexConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_GET_BATCH_SIZE_PROP), HBASE_GET_BATCH_SIZE_PROP,
|
||||
String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_PROP), HBASE_PUT_BATCH_SIZE_PROP,
|
||||
String.valueOf(DEFAULT_HBASE_BATCH_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP),
|
||||
HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE_PROP, DEFAULT_HBASE_PUT_BATCH_SIZE_AUTO_COMPUTE);
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_QPS_FRACTION_PROP), HBASE_QPS_FRACTION_PROP,
|
||||
String.valueOf(DEFAULT_HBASE_QPS_FRACTION));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_MAX_QPS_PER_REGION_SERVER_PROP),
|
||||
HBASE_MAX_QPS_PER_REGION_SERVER_PROP, String.valueOf(DEFAULT_HBASE_MAX_QPS_PER_REGION_SERVER));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY),
|
||||
HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY, String.valueOf(DEFAULT_HOODIE_INDEX_COMPUTE_QPS_DYNAMICALLY));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
|
||||
String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS),
|
||||
HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS, String.valueOf(DEFAULT_HOODIE_INDEX_DESIRED_PUTS_TIME_IN_SECS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_ZK_PATH_QPS_ROOT), HBASE_ZK_PATH_QPS_ROOT,
|
||||
DEFAULT_HBASE_ZK_PATH_QPS_ROOT);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_SESSION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS),
|
||||
HOODIE_INDEX_HBASE_ZK_CONNECTION_TIMEOUT_MS, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_QPS_ALLOCATOR_CLASS), HBASE_INDEX_QPS_ALLOCATOR_CLASS,
|
||||
String.valueOf(DEFAULT_HBASE_INDEX_QPS_ALLOCATOR_CLASS));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_UPDATE_PARTITION_PATH), HBASE_INDEX_UPDATE_PARTITION_PATH,
|
||||
String.valueOf(DEFAULT_HBASE_INDEX_UPDATE_PARTITION_PATH));
|
||||
setDefaultOnCondition(props, !props.containsKey(HBASE_INDEX_ROLLBACK_SYNC), HBASE_INDEX_ROLLBACK_SYNC,
|
||||
String.valueOf(DEFAULT_HBASE_INDEX_ROLLBACK_SYNC));
|
||||
return config;
|
||||
hBaseIndexConfig.setDefaults(HoodieHBaseIndexConfig.class.getName());
|
||||
return hBaseIndexConfig;
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@@ -19,7 +19,8 @@
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.bloom.BloomFilterTypeCode;
|
||||
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.engine.EngineType;
|
||||
import org.apache.hudi.exception.HoodieNotSupportedException;
|
||||
import org.apache.hudi.index.HoodieIndex;
|
||||
@@ -35,60 +36,137 @@ import java.util.Properties;
|
||||
* Indexing related config.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
public class HoodieIndexConfig extends HoodieConfig {
|
||||
|
||||
public static final String INDEX_TYPE_PROP = "hoodie.index.type";
|
||||
public static final ConfigProperty<String> INDEX_TYPE_PROP = ConfigProperty
|
||||
.key("hoodie.index.type")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Type of index to use. Default is Bloom filter. "
|
||||
+ "Possible options are [BLOOM | GLOBAL_BLOOM |SIMPLE | GLOBAL_SIMPLE | INMEMORY | HBASE]. "
|
||||
+ "Bloom filters removes the dependency on a external system "
|
||||
+ "and is stored in the footer of the Parquet Data Files");
|
||||
|
||||
public static final String INDEX_CLASS_PROP = "hoodie.index.class";
|
||||
public static final String DEFAULT_INDEX_CLASS = "";
|
||||
public static final ConfigProperty<String> INDEX_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.index.class")
|
||||
.defaultValue("")
|
||||
.withDocumentation("Full path of user-defined index class and must be a subclass of HoodieIndex class. "
|
||||
+ "It will take precedence over the hoodie.index.type configuration if specified");
|
||||
|
||||
// ***** Bloom Index configs *****
|
||||
public static final String BLOOM_FILTER_NUM_ENTRIES = "hoodie.index.bloom.num_entries";
|
||||
public static final String DEFAULT_BLOOM_FILTER_NUM_ENTRIES = "60000";
|
||||
public static final String BLOOM_FILTER_FPP = "hoodie.index.bloom.fpp";
|
||||
public static final String DEFAULT_BLOOM_FILTER_FPP = "0.000000001";
|
||||
public static final String BLOOM_INDEX_PARALLELISM_PROP = "hoodie.bloom.index.parallelism";
|
||||
// Disable explicit bloom index parallelism setting by default - hoodie auto computes
|
||||
public static final String DEFAULT_BLOOM_INDEX_PARALLELISM = "0";
|
||||
public static final String BLOOM_INDEX_PRUNE_BY_RANGES_PROP = "hoodie.bloom.index.prune.by.ranges";
|
||||
public static final String DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES = "true";
|
||||
public static final String BLOOM_INDEX_USE_CACHING_PROP = "hoodie.bloom.index.use.caching";
|
||||
public static final String DEFAULT_BLOOM_INDEX_USE_CACHING = "true";
|
||||
public static final String BLOOM_INDEX_TREE_BASED_FILTER_PROP = "hoodie.bloom.index.use.treebased.filter";
|
||||
public static final String DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER = "true";
|
||||
public static final ConfigProperty<String> BLOOM_FILTER_NUM_ENTRIES = ConfigProperty
|
||||
.key("hoodie.index.bloom.num_entries")
|
||||
.defaultValue("60000")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "This is the number of entries to be stored in the bloom filter. "
|
||||
+ "We assume the maxParquetFileSize is 128MB and averageRecordSize is 1024B and "
|
||||
+ "hence we approx a total of 130K records in a file. The default (60000) is roughly half of this approximation. "
|
||||
+ "HUDI-56 tracks computing this dynamically. Warning: Setting this very low, "
|
||||
+ "will generate a lot of false positives and index lookup will have to scan a lot more files "
|
||||
+ "than it has to and Setting this to a very high number will increase the size every data file linearly "
|
||||
+ "(roughly 4KB for every 50000 entries). "
|
||||
+ "This config is also used with DYNNAMIC bloom filter which determines the initial size for the bloom.");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_FILTER_FPP = ConfigProperty
|
||||
.key("hoodie.index.bloom.fpp")
|
||||
.defaultValue("0.000000001")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "Error rate allowed given the number of entries. This is used to calculate how many bits should be "
|
||||
+ "assigned for the bloom filter and the number of hash functions. This is usually set very low (default: 0.000000001), "
|
||||
+ "we like to tradeoff disk space for lower false positives. "
|
||||
+ "If the number of entries added to bloom filter exceeds the congfigured value (hoodie.index.bloom.num_entries), "
|
||||
+ "then this fpp may not be honored.");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_PARALLELISM_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.parallelism")
|
||||
.defaultValue("0")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "This is the amount of parallelism for index lookup, which involves a Spark Shuffle. "
|
||||
+ "By default, this is auto computed based on input workload characteristics. "
|
||||
+ "Disable explicit bloom index parallelism setting by default - hoodie auto computes");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_PRUNE_BY_RANGES_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.prune.by.ranges")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "When true, range information from files to leveraged speed up index lookups. Particularly helpful, "
|
||||
+ "if the key has a monotonously increasing prefix, such as timestamp. "
|
||||
+ "If the record key is completely random, it is better to turn this off.");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_USE_CACHING_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.use.caching")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Only applies if index type is BLOOM."
|
||||
+ "When true, the input RDD will cached to speed up index lookup by reducing IO "
|
||||
+ "for computing parallelism or affected partitions");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_TREE_BASED_FILTER_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.use.treebased.filter")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "When true, interval tree based file pruning optimization is enabled. "
|
||||
+ "This mode speeds-up file-pruning based on key ranges when compared with the brute-force mode");
|
||||
|
||||
// TODO: On by default. Once stable, we will remove the other mode.
|
||||
public static final String BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = "hoodie.bloom.index.bucketized.checking";
|
||||
public static final String DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING = "true";
|
||||
public static final String BLOOM_INDEX_FILTER_TYPE = "hoodie.bloom.index.filter.type";
|
||||
public static final String DEFAULT_BLOOM_INDEX_FILTER_TYPE = BloomFilterTypeCode.SIMPLE.name();
|
||||
public static final String HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "hoodie.bloom.index.filter.dynamic.max.entries";
|
||||
public static final String DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = "100000";
|
||||
public static final String SIMPLE_INDEX_USE_CACHING_PROP = "hoodie.simple.index.use.caching";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_USE_CACHING = "true";
|
||||
public static final String SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.simple.index.parallelism";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_PARALLELISM = "50";
|
||||
public static final String GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = "hoodie.global.simple.index.parallelism";
|
||||
public static final String DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM = "100";
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_BUCKETIZED_CHECKING_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.bucketized.checking")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Only applies if index type is BLOOM. "
|
||||
+ "When true, bucketized bloom filtering is enabled. "
|
||||
+ "This reduces skew seen in sort based bloom index lookup");
|
||||
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_FILTER_TYPE = ConfigProperty
|
||||
.key("hoodie.bloom.index.filter.type")
|
||||
.defaultValue(BloomFilterTypeCode.SIMPLE.name())
|
||||
.withDocumentation("Filter type used. Default is BloomFilterTypeCode.SIMPLE. "
|
||||
+ "Available values are [BloomFilterTypeCode.SIMPLE , BloomFilterTypeCode.DYNAMIC_V0]. "
|
||||
+ "Dynamic bloom filters auto size themselves based on number of keys.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES = ConfigProperty
|
||||
.key("hoodie.bloom.index.filter.dynamic.max.entries")
|
||||
.defaultValue("100000")
|
||||
.withDocumentation("The threshold for the maximum number of keys to record in a dynamic Bloom filter row. "
|
||||
+ "Only applies if filter type is BloomFilterTypeCode.DYNAMIC_V0.");
|
||||
|
||||
public static final ConfigProperty<String> SIMPLE_INDEX_USE_CACHING_PROP = ConfigProperty
|
||||
.key("hoodie.simple.index.use.caching")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("Only applies if index type is SIMPLE. "
|
||||
+ "When true, the input RDD will cached to speed up index lookup by reducing IO "
|
||||
+ "for computing parallelism or affected partitions");
|
||||
|
||||
public static final ConfigProperty<String> SIMPLE_INDEX_PARALLELISM_PROP = ConfigProperty
|
||||
.key("hoodie.simple.index.parallelism")
|
||||
.defaultValue("50")
|
||||
.withDocumentation("Only applies if index type is SIMPLE. "
|
||||
+ "This is the amount of parallelism for index lookup, which involves a Spark Shuffle");
|
||||
|
||||
public static final ConfigProperty<String> GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP = ConfigProperty
|
||||
.key("hoodie.global.simple.index.parallelism")
|
||||
.defaultValue("100")
|
||||
.withDocumentation("Only applies if index type is GLOBAL_SIMPLE. "
|
||||
+ "This is the amount of parallelism for index lookup, which involves a Spark Shuffle");
|
||||
|
||||
// 1B bloom filter checks happen in 250 seconds. 500ms to read a bloom filter.
|
||||
// 10M checks in 2500ms, thus amortizing the cost of reading bloom filter across partitions.
|
||||
public static final String BLOOM_INDEX_KEYS_PER_BUCKET_PROP = "hoodie.bloom.index.keys.per.bucket";
|
||||
public static final String DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET = "10000000";
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_KEYS_PER_BUCKET_PROP = ConfigProperty
|
||||
.key("hoodie.bloom.index.keys.per.bucket")
|
||||
.defaultValue("10000000")
|
||||
.withDocumentation("Only applies if bloomIndexBucketizedChecking is enabled and index type is bloom. "
|
||||
+ "This configuration controls the “bucket” size which tracks the number of record-key checks made against "
|
||||
+ "a single file and is the unit of work allocated to each partition performing bloom filter lookup. "
|
||||
+ "A higher value would amortize the fixed cost of reading a bloom filter to memory.");
|
||||
|
||||
// ***** HBase Index Configs *****
|
||||
public static final String HBASE_ZKQUORUM_PROP = "hoodie.index.hbase.zkquorum";
|
||||
public static final String HBASE_ZKPORT_PROP = "hoodie.index.hbase.zkport";
|
||||
public static final String HBASE_ZK_ZNODEPARENT = "hoodie.index.hbase.zknode.path";
|
||||
public static final String HBASE_TABLENAME_PROP = "hoodie.index.hbase.table";
|
||||
public static final String HBASE_GET_BATCH_SIZE_PROP = "hoodie.index.hbase.get.batch.size";
|
||||
public static final String HBASE_PUT_BATCH_SIZE_PROP = "hoodie.index.hbase.put.batch.size";
|
||||
public static final String DEFAULT_HBASE_BATCH_SIZE = "100";
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_INPUT_STORAGE_LEVEL = ConfigProperty
|
||||
.key("hoodie.bloom.index.input.storage.level")
|
||||
.defaultValue("MEMORY_AND_DISK_SER")
|
||||
.withDocumentation("Only applies when #bloomIndexUseCaching is set. Determine what level of persistence is used to cache input RDDs. "
|
||||
+ "Refer to org.apache.spark.storage.StorageLevel for different values");
|
||||
|
||||
|
||||
public static final String BLOOM_INDEX_INPUT_STORAGE_LEVEL = "hoodie.bloom.index.input.storage.level";
|
||||
public static final String DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
public static final String SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "hoodie.simple.index.input.storage.level";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL = "MEMORY_AND_DISK_SER";
|
||||
public static final ConfigProperty<String> SIMPLE_INDEX_INPUT_STORAGE_LEVEL = ConfigProperty
|
||||
.key("hoodie.simple.index.input.storage.level")
|
||||
.defaultValue("MEMORY_AND_DISK_SER")
|
||||
.withDocumentation("Only applies when #simpleIndexUseCaching is set. Determine what level of persistence is used to cache input RDDs. "
|
||||
+ "Refer to org.apache.spark.storage.StorageLevel for different values");
|
||||
|
||||
/**
|
||||
* Only applies if index type is GLOBAL_BLOOM.
|
||||
@@ -98,23 +176,31 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
* <p>
|
||||
* When set to false, a record will be updated to the old partition.
|
||||
*/
|
||||
public static final String BLOOM_INDEX_UPDATE_PARTITION_PATH = "hoodie.bloom.index.update.partition.path";
|
||||
public static final String DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
public static final ConfigProperty<String> BLOOM_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
|
||||
.key("hoodie.bloom.index.update.partition.path")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Only applies if index type is GLOBAL_BLOOM. "
|
||||
+ "When set to true, an update including the partition path of a record that already exists will result in "
|
||||
+ "inserting the incoming record into the new partition and deleting the original record in the old partition. "
|
||||
+ "When set to false, the original record will only be updated in the old partition");
|
||||
|
||||
public static final String SIMPLE_INDEX_UPDATE_PARTITION_PATH = "hoodie.simple.index.update.partition.path";
|
||||
public static final String DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH = "false";
|
||||
public static final ConfigProperty<String> SIMPLE_INDEX_UPDATE_PARTITION_PATH = ConfigProperty
|
||||
.key("hoodie.simple.index.update.partition.path")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("");
|
||||
|
||||
private EngineType engineType;
|
||||
|
||||
/**
|
||||
* Use Spark engine by default.
|
||||
*/
|
||||
private HoodieIndexConfig(Properties props) {
|
||||
this(EngineType.SPARK, props);
|
||||
|
||||
private HoodieIndexConfig() {
|
||||
this(EngineType.SPARK);
|
||||
}
|
||||
|
||||
private HoodieIndexConfig(EngineType engineType, Properties props) {
|
||||
super(props);
|
||||
private HoodieIndexConfig(EngineType engineType) {
|
||||
super();
|
||||
this.engineType = engineType;
|
||||
}
|
||||
|
||||
@@ -125,127 +211,107 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
public static class Builder {
|
||||
|
||||
private EngineType engineType = EngineType.SPARK;
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieIndexConfig hoodieIndexConfig = new HoodieIndexConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.hoodieIndexConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.hoodieIndexConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexType(HoodieIndex.IndexType indexType) {
|
||||
props.setProperty(INDEX_TYPE_PROP, indexType.name());
|
||||
hoodieIndexConfig.setValue(INDEX_TYPE_PROP, indexType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIndexClass(String indexClass) {
|
||||
props.setProperty(INDEX_CLASS_PROP, indexClass);
|
||||
hoodieIndexConfig.setValue(INDEX_CLASS_PROP, indexClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withHBaseIndexConfig(HoodieHBaseIndexConfig hBaseIndexConfig) {
|
||||
props.putAll(hBaseIndexConfig.getProps());
|
||||
hoodieIndexConfig.getProps().putAll(hBaseIndexConfig.getProps());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterNumEntries(int numEntries) {
|
||||
props.setProperty(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
||||
hoodieIndexConfig.setValue(BLOOM_FILTER_NUM_ENTRIES, String.valueOf(numEntries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomFilterFPP(double fpp) {
|
||||
props.setProperty(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkQuorum(String zkString) {
|
||||
props.setProperty(HBASE_ZKQUORUM_PROP, zkString);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkPort(int port) {
|
||||
props.setProperty(HBASE_ZKPORT_PROP, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseZkZnodeParent(String zkZnodeParent) {
|
||||
props.setProperty(HBASE_ZK_ZNODEPARENT, zkZnodeParent);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hbaseTableName(String tableName) {
|
||||
props.setProperty(HBASE_TABLENAME_PROP, tableName);
|
||||
hoodieIndexConfig.setValue(BLOOM_FILTER_FPP, String.valueOf(fpp));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexParallelism(int parallelism) {
|
||||
props.setProperty(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexPruneByRanges(boolean pruneRanges) {
|
||||
props.setProperty(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_PRUNE_BY_RANGES_PROP, String.valueOf(pruneRanges));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexTreebasedFilter(boolean useTreeFilter) {
|
||||
props.setProperty(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_TREE_BASED_FILTER_PROP, String.valueOf(useTreeFilter));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexBucketizedChecking(boolean bucketizedChecking) {
|
||||
props.setProperty(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, String.valueOf(bucketizedChecking));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder bloomIndexKeysPerBucket(int keysPerBucket) {
|
||||
props.setProperty(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_KEYS_PER_BUCKET_PROP, String.valueOf(keysPerBucket));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBloomIndexInputStorageLevel(String level) {
|
||||
props.setProperty(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBloomIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
hoodieIndexConfig.setValue(BLOOM_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
hoodieIndexConfig.setValue(SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder simpleIndexUseCaching(boolean useCaching) {
|
||||
props.setProperty(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
hoodieIndexConfig.setValue(SIMPLE_INDEX_USE_CACHING_PROP, String.valueOf(useCaching));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSimpleIndexInputStorageLevel(String level) {
|
||||
props.setProperty(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
hoodieIndexConfig.setValue(SIMPLE_INDEX_INPUT_STORAGE_LEVEL, level);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexParallelism(int parallelism) {
|
||||
props.setProperty(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
hoodieIndexConfig.setValue(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withGlobalSimpleIndexUpdatePartitionPath(boolean updatePartitionPath) {
|
||||
props.setProperty(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
hoodieIndexConfig.setValue(SIMPLE_INDEX_UPDATE_PARTITION_PATH, String.valueOf(updatePartitionPath));
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -255,45 +321,12 @@ public class HoodieIndexConfig extends DefaultHoodieConfig {
|
||||
}
|
||||
|
||||
public HoodieIndexConfig build() {
|
||||
HoodieIndexConfig config = new HoodieIndexConfig(engineType, props);
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_TYPE_PROP), INDEX_TYPE_PROP, getDefaultIndexType(engineType));
|
||||
setDefaultOnCondition(props, !props.containsKey(INDEX_CLASS_PROP), INDEX_CLASS_PROP, DEFAULT_INDEX_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_NUM_ENTRIES), BLOOM_FILTER_NUM_ENTRIES,
|
||||
DEFAULT_BLOOM_FILTER_NUM_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_FILTER_FPP), BLOOM_FILTER_FPP, DEFAULT_BLOOM_FILTER_FPP);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PARALLELISM_PROP), BLOOM_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_BLOOM_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_PRUNE_BY_RANGES_PROP),
|
||||
BLOOM_INDEX_PRUNE_BY_RANGES_PROP, DEFAULT_BLOOM_INDEX_PRUNE_BY_RANGES);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_USE_CACHING_PROP), BLOOM_INDEX_USE_CACHING_PROP,
|
||||
DEFAULT_BLOOM_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_INPUT_STORAGE_LEVEL), BLOOM_INDEX_INPUT_STORAGE_LEVEL,
|
||||
DEFAULT_BLOOM_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_UPDATE_PARTITION_PATH),
|
||||
BLOOM_INDEX_UPDATE_PARTITION_PATH, DEFAULT_BLOOM_INDEX_UPDATE_PARTITION_PATH);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_TREE_BASED_FILTER_PROP),
|
||||
BLOOM_INDEX_TREE_BASED_FILTER_PROP, DEFAULT_BLOOM_INDEX_TREE_BASED_FILTER);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_BUCKETIZED_CHECKING_PROP),
|
||||
BLOOM_INDEX_BUCKETIZED_CHECKING_PROP, DEFAULT_BLOOM_INDEX_BUCKETIZED_CHECKING);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_KEYS_PER_BUCKET_PROP),
|
||||
BLOOM_INDEX_KEYS_PER_BUCKET_PROP, DEFAULT_BLOOM_INDEX_KEYS_PER_BUCKET);
|
||||
setDefaultOnCondition(props, !props.containsKey(BLOOM_INDEX_FILTER_TYPE),
|
||||
BLOOM_INDEX_FILTER_TYPE, DEFAULT_BLOOM_INDEX_FILTER_TYPE);
|
||||
setDefaultOnCondition(props, !props.containsKey(HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES),
|
||||
HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES, DEFAULT_HOODIE_BLOOM_INDEX_FILTER_DYNAMIC_MAX_ENTRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_PARALLELISM_PROP), SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_USE_CACHING_PROP), SIMPLE_INDEX_USE_CACHING_PROP,
|
||||
DEFAULT_SIMPLE_INDEX_USE_CACHING);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_INPUT_STORAGE_LEVEL), SIMPLE_INDEX_INPUT_STORAGE_LEVEL,
|
||||
DEFAULT_SIMPLE_INDEX_INPUT_STORAGE_LEVEL);
|
||||
setDefaultOnCondition(props, !props.containsKey(GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP), GLOBAL_SIMPLE_INDEX_PARALLELISM_PROP,
|
||||
DEFAULT_GLOBAL_SIMPLE_INDEX_PARALLELISM);
|
||||
setDefaultOnCondition(props, !props.containsKey(SIMPLE_INDEX_UPDATE_PARTITION_PATH),
|
||||
SIMPLE_INDEX_UPDATE_PARTITION_PATH, DEFAULT_SIMPLE_INDEX_UPDATE_PARTITION_PATH);
|
||||
hoodieIndexConfig.setDefaultValue(INDEX_TYPE_PROP, getDefaultIndexType(engineType));
|
||||
hoodieIndexConfig.setDefaults(HoodieIndexConfig.class.getName());
|
||||
|
||||
// Throws IllegalArgumentException if the value set is not a known Hoodie Index Type
|
||||
HoodieIndex.IndexType.valueOf(props.getProperty(INDEX_TYPE_PROP));
|
||||
return config;
|
||||
HoodieIndex.IndexType.valueOf(hoodieIndexConfig.getString(INDEX_TYPE_PROP));
|
||||
return hoodieIndexConfig;
|
||||
}
|
||||
|
||||
private String getDefaultIndexType(EngineType engineType) {
|
||||
|
||||
@@ -20,7 +20,8 @@ package org.apache.hudi.config;
|
||||
import org.apache.hudi.client.transaction.SimpleConcurrentFileWritesConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.ConflictResolutionStrategy;
|
||||
import org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider;
|
||||
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.lock.LockProvider;
|
||||
|
||||
import java.io.File;
|
||||
@@ -28,48 +29,152 @@ import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_NUM_RETRIES;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_CONNECTION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.DEFAULT_ZK_SESSION_TIMEOUT_MS;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_DATABASE_NAME_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_METASTORE_URI_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.HIVE_TABLE_NAME_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.LOCK_PREFIX;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_BASE_PATH_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_PORT_PROP_KEY;
|
||||
import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY;
|
||||
|
||||
|
||||
/**
|
||||
* Hoodie Configs for Locks.
|
||||
*/
|
||||
public class HoodieLockConfig extends DefaultHoodieConfig {
|
||||
public class HoodieLockConfig extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<String> LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)
|
||||
.defaultValue(DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS)
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Parameter used in the exponential backoff retry policy. Stands for the Initial amount "
|
||||
+ "of time to wait between retries by lock provider client");
|
||||
|
||||
public static final ConfigProperty<String> LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY)
|
||||
.defaultValue(String.valueOf(5000L))
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Parameter used in the exponential backoff retry policy. Stands for the maximum amount "
|
||||
+ "of time to wait between retries by lock provider client");
|
||||
|
||||
public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY)
|
||||
.defaultValue(String.valueOf(10000L))
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Amount of time to wait between retries from the hudi client");
|
||||
|
||||
public static final ConfigProperty<String> LOCK_ACQUIRE_NUM_RETRIES_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY)
|
||||
.defaultValue(DEFAULT_LOCK_ACQUIRE_NUM_RETRIES)
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Maximum number of times to retry by lock provider client");
|
||||
|
||||
public static final ConfigProperty<String> LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY)
|
||||
.defaultValue(String.valueOf(0))
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Maximum number of times to retry to acquire lock additionally from the hudi client");
|
||||
|
||||
public static final ConfigProperty<Integer> LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = ConfigProperty
|
||||
.key(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY)
|
||||
.defaultValue(60 * 1000)
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> FILESYSTEM_LOCK_PATH_PROP = ConfigProperty
|
||||
.key(FILESYSTEM_LOCK_PATH_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_DATABASE_NAME_PROP = ConfigProperty
|
||||
.key(HIVE_DATABASE_NAME_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("The Hive database to acquire lock against");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_TABLE_NAME_PROP = ConfigProperty
|
||||
.key(HIVE_TABLE_NAME_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("The Hive table under the hive database to acquire lock against");
|
||||
|
||||
public static final ConfigProperty<String> HIVE_METASTORE_URI_PROP = ConfigProperty
|
||||
.key(HIVE_METASTORE_URI_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> ZK_BASE_PATH_PROP = ConfigProperty
|
||||
.key(ZK_BASE_PATH_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("The base path on Zookeeper under which to create a ZNode to acquire the lock. "
|
||||
+ "This should be common for all jobs writing to the same table");
|
||||
|
||||
public static final ConfigProperty<Integer> ZK_SESSION_TIMEOUT_MS_PROP = ConfigProperty
|
||||
.key(ZK_SESSION_TIMEOUT_MS_PROP_KEY)
|
||||
.defaultValue(DEFAULT_ZK_SESSION_TIMEOUT_MS)
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("How long to wait after losing a connection to ZooKeeper before the session is expired");
|
||||
|
||||
public static final ConfigProperty<Integer> ZK_CONNECTION_TIMEOUT_MS_PROP = ConfigProperty
|
||||
.key(ZK_CONNECTION_TIMEOUT_MS_PROP_KEY)
|
||||
.defaultValue(DEFAULT_ZK_CONNECTION_TIMEOUT_MS)
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("How long to wait when connecting to ZooKeeper before considering the connection a failure");
|
||||
|
||||
public static final ConfigProperty<String> ZK_CONNECT_URL_PROP = ConfigProperty
|
||||
.key(ZK_CONNECT_URL_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Set the list of comma separated servers to connect to");
|
||||
|
||||
public static final ConfigProperty<String> ZK_PORT_PROP = ConfigProperty
|
||||
.key(ZK_PORT_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("The connection port to be used for Zookeeper");
|
||||
|
||||
public static final ConfigProperty<String> ZK_LOCK_KEY_PROP = ConfigProperty
|
||||
.key(ZK_LOCK_KEY_PROP_KEY)
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Key name under base_path at which to create a ZNode and acquire lock. "
|
||||
+ "Final path on zk will look like base_path/lock_key. We recommend setting this to the table name");
|
||||
|
||||
// Pluggable type of lock provider
|
||||
public static final String LOCK_PROVIDER_CLASS_PROP = LOCK_PREFIX + "provider";
|
||||
public static final String DEFAULT_LOCK_PROVIDER_CLASS = ZookeeperBasedLockProvider.class.getName();
|
||||
// Pluggable strategies to use when resolving conflicts
|
||||
public static final String WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP =
|
||||
LOCK_PREFIX + "conflict.resolution.strategy";
|
||||
public static final String DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS =
|
||||
SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName();
|
||||
public static final ConfigProperty<String> LOCK_PROVIDER_CLASS_PROP = ConfigProperty
|
||||
.key(LOCK_PREFIX + "provider")
|
||||
.defaultValue(ZookeeperBasedLockProvider.class.getName())
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Lock provider class name, user can provide their own implementation of LockProvider "
|
||||
+ "which should be subclass of org.apache.hudi.common.lock.LockProvider");
|
||||
|
||||
private HoodieLockConfig(Properties props) {
|
||||
super(props);
|
||||
// Pluggable strategies to use when resolving conflicts
|
||||
public static final ConfigProperty<String> WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP = ConfigProperty
|
||||
.key(LOCK_PREFIX + "conflict.resolution.strategy")
|
||||
.defaultValue(SimpleConcurrentFileWritesConflictResolutionStrategy.class.getName())
|
||||
.sinceVersion("0.8.0")
|
||||
.withDocumentation("Lock provider class name, this should be subclass of "
|
||||
+ "org.apache.hudi.client.transaction.ConflictResolutionStrategy");
|
||||
|
||||
private HoodieLockConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieLockConfig.Builder newBuilder() {
|
||||
@@ -78,128 +183,108 @@ public class HoodieLockConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieLockConfig lockConfig = new HoodieLockConfig();
|
||||
|
||||
public HoodieLockConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.lockConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.lockConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withLockProvider(Class<? extends LockProvider> lockProvider) {
|
||||
props.setProperty(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName());
|
||||
lockConfig.setValue(LOCK_PROVIDER_CLASS_PROP, lockProvider.getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withHiveDatabaseName(String databaseName) {
|
||||
props.setProperty(HIVE_DATABASE_NAME_PROP, databaseName);
|
||||
lockConfig.setValue(HIVE_DATABASE_NAME_PROP, databaseName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withHiveTableName(String tableName) {
|
||||
props.setProperty(HIVE_TABLE_NAME_PROP, tableName);
|
||||
lockConfig.setValue(HIVE_TABLE_NAME_PROP, tableName);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withHiveMetastoreURIs(String hiveMetastoreURIs) {
|
||||
props.setProperty(HIVE_METASTORE_URI_PROP, hiveMetastoreURIs);
|
||||
lockConfig.setValue(HIVE_METASTORE_URI_PROP, hiveMetastoreURIs);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkQuorum(String zkQuorum) {
|
||||
props.setProperty(ZK_CONNECT_URL_PROP, zkQuorum);
|
||||
lockConfig.setValue(ZK_CONNECT_URL_PROP, zkQuorum);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkBasePath(String zkBasePath) {
|
||||
props.setProperty(ZK_BASE_PATH_PROP, zkBasePath);
|
||||
lockConfig.setValue(ZK_BASE_PATH_PROP, zkBasePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkPort(String zkPort) {
|
||||
props.setProperty(ZK_PORT_PROP, zkPort);
|
||||
lockConfig.setValue(ZK_PORT_PROP, zkPort);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkLockKey(String zkLockKey) {
|
||||
props.setProperty(ZK_LOCK_KEY_PROP, zkLockKey);
|
||||
lockConfig.setValue(ZK_LOCK_KEY_PROP, zkLockKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkConnectionTimeoutInMs(Long connectionTimeoutInMs) {
|
||||
props.setProperty(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs));
|
||||
lockConfig.setValue(ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(connectionTimeoutInMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withZkSessionTimeoutInMs(Long sessionTimeoutInMs) {
|
||||
props.setProperty(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs));
|
||||
lockConfig.setValue(ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(sessionTimeoutInMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withNumRetries(int numRetries) {
|
||||
props.setProperty(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_NUM_RETRIES_PROP, String.valueOf(numRetries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withRetryWaitTimeInMillis(Long retryWaitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryWaitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withRetryMaxWaitTimeInMillis(Long retryMaxWaitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryMaxWaitTimeInMillis));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(retryMaxWaitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withClientNumRetries(int clientNumRetries) {
|
||||
props.setProperty(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, String.valueOf(clientNumRetries));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withClientRetryWaitTimeInMillis(Long clientRetryWaitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, String.valueOf(clientRetryWaitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withLockWaitTimeInMillis(Long waitTimeInMillis) {
|
||||
props.setProperty(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis));
|
||||
lockConfig.setValue(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(waitTimeInMillis));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig.Builder withConflictResolutionStrategy(ConflictResolutionStrategy conflictResolutionStrategy) {
|
||||
props.setProperty(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName());
|
||||
lockConfig.setValue(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, conflictResolutionStrategy.getClass().getName());
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieLockConfig build() {
|
||||
HoodieLockConfig config = new HoodieLockConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_PROVIDER_CLASS_PROP),
|
||||
LOCK_PROVIDER_CLASS_PROP, DEFAULT_LOCK_PROVIDER_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP),
|
||||
WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS_PROP, DEFAULT_WRITE_CONFLICT_RESOLUTION_STRATEGY_CLASS);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_NUM_RETRIES_PROP),
|
||||
LOCK_ACQUIRE_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_NUM_RETRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP),
|
||||
LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP),
|
||||
LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP),
|
||||
LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP, DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS);
|
||||
setDefaultOnCondition(props, !props.containsKey(ZK_CONNECTION_TIMEOUT_MS_PROP),
|
||||
ZK_CONNECTION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_CONNECTION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(ZK_SESSION_TIMEOUT_MS_PROP),
|
||||
ZK_SESSION_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ZK_SESSION_TIMEOUT_MS));
|
||||
setDefaultOnCondition(props, !props.containsKey(LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP),
|
||||
LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP, String.valueOf(DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS));
|
||||
return config;
|
||||
lockConfig.setDefaults(HoodieLockConfig.class.getName());
|
||||
return lockConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
@@ -31,39 +32,59 @@ import java.util.Properties;
|
||||
* Memory related config.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
public class HoodieMemoryConfig extends HoodieConfig {
|
||||
|
||||
// This fraction is multiplied with the spark.memory.fraction to get a final fraction of heap space to use
|
||||
// during merge. This makes it easier to scale this value as one increases the spark.executor.memory
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_MERGE_PROP = "hoodie.memory.merge.fraction";
|
||||
// Default max memory fraction during hash-merge, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE = String.valueOf(0.6);
|
||||
public static final String MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = "hoodie.memory.compaction.fraction";
|
||||
public static final ConfigProperty<String> MAX_MEMORY_FRACTION_FOR_MERGE_PROP = ConfigProperty
|
||||
.key("hoodie.memory.merge.fraction")
|
||||
.defaultValue(String.valueOf(0.6))
|
||||
.withDocumentation("This fraction is multiplied with the user memory fraction (1 - spark.memory.fraction) "
|
||||
+ "to get a final fraction of heap space to use during merge");
|
||||
|
||||
// Default max memory fraction during compaction, excess spills to disk
|
||||
public static final String DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION = String.valueOf(0.6);
|
||||
public static final ConfigProperty<String> MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP = ConfigProperty
|
||||
.key("hoodie.memory.compaction.fraction")
|
||||
.defaultValue(String.valueOf(0.6))
|
||||
.withDocumentation("HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then "
|
||||
+ "merges these log blocks and records. At any point, the number of entries in a log block can be "
|
||||
+ "less than or equal to the number of entries in the corresponding parquet file. This can lead to "
|
||||
+ "OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to "
|
||||
+ "set the max allowable inMemory footprint of the spillable map");
|
||||
|
||||
// Default memory size (1GB) per compaction (used if SparkEnv is absent), excess spills to disk
|
||||
public static final long DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 1024 * 1024 * 1024L;
|
||||
// Minimum memory size (100MB) for the spillable map.
|
||||
public static final long DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES = 100 * 1024 * 1024L;
|
||||
// Property to set the max memory for merge
|
||||
public static final String MAX_MEMORY_FOR_MERGE_PROP = "hoodie.memory.merge.max.size";
|
||||
// Property to set the max memory for compaction
|
||||
public static final String MAX_MEMORY_FOR_COMPACTION_PROP = "hoodie.memory.compaction.max.size";
|
||||
// Property to set the max memory for dfs inputstream buffer size
|
||||
public static final String MAX_DFS_STREAM_BUFFER_SIZE_PROP = "hoodie.memory.dfs.buffer.max.size";
|
||||
public static final int DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE = 16 * 1024 * 1024; // 16MB
|
||||
public static final String SPILLABLE_MAP_BASE_PATH_PROP = "hoodie.memory.spillable.map.path";
|
||||
// Default file path prefix for spillable file
|
||||
public static final String DEFAULT_SPILLABLE_MAP_BASE_PATH = "/tmp/";
|
||||
|
||||
// Property to control how what fraction of the failed record, exceptions we report back to driver.
|
||||
public static final String WRITESTATUS_FAILURE_FRACTION_PROP = "hoodie.memory.writestatus.failure.fraction";
|
||||
// Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and
|
||||
// mask actual data errors.
|
||||
public static final double DEFAULT_WRITESTATUS_FAILURE_FRACTION = 0.1;
|
||||
public static final ConfigProperty<Long> MAX_MEMORY_FOR_MERGE_PROP = ConfigProperty
|
||||
.key("hoodie.memory.merge.max.size")
|
||||
.defaultValue(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)
|
||||
.withDocumentation("Property to set the max memory for merge");
|
||||
|
||||
private HoodieMemoryConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> MAX_MEMORY_FOR_COMPACTION_PROP = ConfigProperty
|
||||
.key("hoodie.memory.compaction.max.size")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Property to set the max memory for compaction");
|
||||
|
||||
public static final ConfigProperty<Integer> MAX_DFS_STREAM_BUFFER_SIZE_PROP = ConfigProperty
|
||||
.key("hoodie.memory.dfs.buffer.max.size")
|
||||
.defaultValue(16 * 1024 * 1024)
|
||||
.withDocumentation("Property to set the max memory for dfs inputstream buffer size");
|
||||
|
||||
public static final ConfigProperty<String> SPILLABLE_MAP_BASE_PATH_PROP = ConfigProperty
|
||||
.key("hoodie.memory.spillable.map.path")
|
||||
.defaultValue("/tmp/")
|
||||
.withDocumentation("Default file path prefix for spillable file");
|
||||
|
||||
public static final ConfigProperty<Double> WRITESTATUS_FAILURE_FRACTION_PROP = ConfigProperty
|
||||
.key("hoodie.memory.writestatus.failure.fraction")
|
||||
.defaultValue(0.1)
|
||||
.withDocumentation("Property to control how what fraction of the failed record, exceptions we report back to driver. "
|
||||
+ "Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and "
|
||||
+ "mask actual data errors.");
|
||||
|
||||
private HoodieMemoryConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieMemoryConfig.Builder newBuilder() {
|
||||
@@ -72,57 +93,49 @@ public class HoodieMemoryConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieMemoryConfig memoryConfig = new HoodieMemoryConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.memoryConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.memoryConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerPartitionMerge(double maxMemoryFractionPerPartitionMerge) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge));
|
||||
memoryConfig.setValue(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, String.valueOf(maxMemoryFractionPerPartitionMerge));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryMaxSize(long mergeMaxSize, long compactionMaxSize) {
|
||||
props.setProperty(MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(mergeMaxSize));
|
||||
props.setProperty(MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(compactionMaxSize));
|
||||
memoryConfig.setValue(MAX_MEMORY_FOR_MERGE_PROP, String.valueOf(mergeMaxSize));
|
||||
memoryConfig.setValue(MAX_MEMORY_FOR_COMPACTION_PROP, String.valueOf(compactionMaxSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryFractionPerCompaction(double maxMemoryFractionPerCompaction) {
|
||||
props.setProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction));
|
||||
memoryConfig.setValue(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, String.valueOf(maxMemoryFractionPerCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxDFSStreamBufferSize(int maxStreamBufferSize) {
|
||||
props.setProperty(MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(maxStreamBufferSize));
|
||||
memoryConfig.setValue(MAX_DFS_STREAM_BUFFER_SIZE_PROP, String.valueOf(maxStreamBufferSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withWriteStatusFailureFraction(double failureFraction) {
|
||||
props.setProperty(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
|
||||
memoryConfig.setValue(WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(failureFraction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMemoryConfig build() {
|
||||
HoodieMemoryConfig config = new HoodieMemoryConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_DFS_STREAM_BUFFER_SIZE_PROP), MAX_DFS_STREAM_BUFFER_SIZE_PROP,
|
||||
String.valueOf(DEFAULT_MAX_DFS_STREAM_BUFFER_SIZE));
|
||||
setDefaultOnCondition(props, !props.containsKey(SPILLABLE_MAP_BASE_PATH_PROP), SPILLABLE_MAP_BASE_PATH_PROP,
|
||||
DEFAULT_SPILLABLE_MAP_BASE_PATH);
|
||||
setDefaultOnCondition(props, !props.containsKey(MAX_MEMORY_FOR_MERGE_PROP), MAX_MEMORY_FOR_MERGE_PROP,
|
||||
String.valueOf(DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES));
|
||||
setDefaultOnCondition(props, !props.containsKey(WRITESTATUS_FAILURE_FRACTION_PROP),
|
||||
WRITESTATUS_FAILURE_FRACTION_PROP, String.valueOf(DEFAULT_WRITESTATUS_FAILURE_FRACTION));
|
||||
return config;
|
||||
memoryConfig.setDefaults(HoodieMemoryConfig.class.getName());
|
||||
return memoryConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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.metrics.MetricsReporterType;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
@@ -32,41 +33,74 @@ import java.util.Properties;
|
||||
* Fetch the configurations used by the Metrics system.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
public class HoodieMetricsConfig extends HoodieConfig {
|
||||
|
||||
public static final String METRIC_PREFIX = "hoodie.metrics";
|
||||
public static final String METRICS_ON = METRIC_PREFIX + ".on";
|
||||
public static final boolean DEFAULT_METRICS_ON = false;
|
||||
public static final String METRICS_REPORTER_TYPE = METRIC_PREFIX + ".reporter.type";
|
||||
public static final MetricsReporterType DEFAULT_METRICS_REPORTER_TYPE = MetricsReporterType.GRAPHITE;
|
||||
|
||||
public static final ConfigProperty<Boolean> METRICS_ON = ConfigProperty
|
||||
.key(METRIC_PREFIX + ".on")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("Turn on/off metrics reporting. off by default.");
|
||||
|
||||
public static final ConfigProperty<MetricsReporterType> METRICS_REPORTER_TYPE = ConfigProperty
|
||||
.key(METRIC_PREFIX + ".reporter.type")
|
||||
.defaultValue(MetricsReporterType.GRAPHITE)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("Type of metrics reporter.");
|
||||
|
||||
// Graphite
|
||||
public static final String GRAPHITE_PREFIX = METRIC_PREFIX + ".graphite";
|
||||
public static final String GRAPHITE_SERVER_HOST = GRAPHITE_PREFIX + ".host";
|
||||
public static final String DEFAULT_GRAPHITE_SERVER_HOST = "localhost";
|
||||
|
||||
public static final String GRAPHITE_SERVER_PORT = GRAPHITE_PREFIX + ".port";
|
||||
public static final int DEFAULT_GRAPHITE_SERVER_PORT = 4756;
|
||||
public static final ConfigProperty<String> GRAPHITE_SERVER_HOST = ConfigProperty
|
||||
.key(GRAPHITE_PREFIX + ".host")
|
||||
.defaultValue("localhost")
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("Graphite host to connect to");
|
||||
|
||||
public static final ConfigProperty<Integer> GRAPHITE_SERVER_PORT = ConfigProperty
|
||||
.key(GRAPHITE_PREFIX + ".port")
|
||||
.defaultValue(4756)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("Graphite port to connect to");
|
||||
|
||||
// Jmx
|
||||
public static final String JMX_PREFIX = METRIC_PREFIX + ".jmx";
|
||||
public static final String JMX_HOST = JMX_PREFIX + ".host";
|
||||
public static final String DEFAULT_JMX_HOST = "localhost";
|
||||
|
||||
public static final String JMX_PORT = JMX_PREFIX + ".port";
|
||||
public static final int DEFAULT_JMX_PORT = 9889;
|
||||
public static final ConfigProperty<String> JMX_HOST = ConfigProperty
|
||||
.key(JMX_PREFIX + ".host")
|
||||
.defaultValue("localhost")
|
||||
.sinceVersion("0.5.1")
|
||||
.withDocumentation("Jmx host to connect to");
|
||||
|
||||
public static final String GRAPHITE_METRIC_PREFIX = GRAPHITE_PREFIX + ".metric.prefix";
|
||||
public static final ConfigProperty<Integer> JMX_PORT = ConfigProperty
|
||||
.key(JMX_PREFIX + ".port")
|
||||
.defaultValue(9889)
|
||||
.sinceVersion("0.5.1")
|
||||
.withDocumentation("Jmx port to connect to");
|
||||
|
||||
public static final ConfigProperty<String> GRAPHITE_METRIC_PREFIX = ConfigProperty
|
||||
.key(GRAPHITE_PREFIX + ".metric.prefix")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.5.1")
|
||||
.withDocumentation("Standard prefix applied to all metrics. This helps to add datacenter, environment information for e.g");
|
||||
|
||||
// User defined
|
||||
public static final String METRICS_REPORTER_CLASS = METRIC_PREFIX + ".reporter.class";
|
||||
public static final String DEFAULT_METRICS_REPORTER_CLASS = "";
|
||||
public static final ConfigProperty<String> METRICS_REPORTER_CLASS = ConfigProperty
|
||||
.key(METRIC_PREFIX + ".reporter.class")
|
||||
.defaultValue("")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// Enable metrics collection from executors
|
||||
public static final String ENABLE_EXECUTOR_METRICS = METRIC_PREFIX + ".executor.enable";
|
||||
public static final ConfigProperty<String> ENABLE_EXECUTOR_METRICS = ConfigProperty
|
||||
.key(METRIC_PREFIX + ".executor.enable")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
|
||||
private HoodieMetricsConfig(Properties props) {
|
||||
super(props);
|
||||
private HoodieMetricsConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieMetricsConfig.Builder newBuilder() {
|
||||
@@ -75,89 +109,78 @@ public class HoodieMetricsConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieMetricsConfig hoodieMetricsConfig = new HoodieMetricsConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.hoodieMetricsConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.hoodieMetricsConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder on(boolean metricsOn) {
|
||||
props.setProperty(METRICS_ON, String.valueOf(metricsOn));
|
||||
hoodieMetricsConfig.setValue(METRICS_ON, String.valueOf(metricsOn));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withReporterType(String reporterType) {
|
||||
props.setProperty(METRICS_REPORTER_TYPE, reporterType);
|
||||
hoodieMetricsConfig.setValue(METRICS_REPORTER_TYPE, reporterType);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder toGraphiteHost(String host) {
|
||||
props.setProperty(GRAPHITE_SERVER_HOST, host);
|
||||
hoodieMetricsConfig.setValue(GRAPHITE_SERVER_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder onGraphitePort(int port) {
|
||||
props.setProperty(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
||||
hoodieMetricsConfig.setValue(GRAPHITE_SERVER_PORT, String.valueOf(port));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder toJmxHost(String host) {
|
||||
props.setProperty(JMX_HOST, host);
|
||||
hoodieMetricsConfig.setValue(JMX_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder onJmxPort(String port) {
|
||||
props.setProperty(JMX_PORT, port);
|
||||
hoodieMetricsConfig.setValue(JMX_PORT, port);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder usePrefix(String prefix) {
|
||||
props.setProperty(GRAPHITE_METRIC_PREFIX, prefix);
|
||||
hoodieMetricsConfig.setValue(GRAPHITE_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withReporterClass(String className) {
|
||||
props.setProperty(METRICS_REPORTER_CLASS, className);
|
||||
hoodieMetricsConfig.setValue(METRICS_REPORTER_CLASS, className);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withExecutorMetrics(boolean enable) {
|
||||
props.setProperty(ENABLE_EXECUTOR_METRICS, String.valueOf(enable));
|
||||
hoodieMetricsConfig.setValue(ENABLE_EXECUTOR_METRICS, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsConfig build() {
|
||||
HoodieMetricsConfig config = new HoodieMetricsConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_ON), METRICS_ON, String.valueOf(DEFAULT_METRICS_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_TYPE), METRICS_REPORTER_TYPE,
|
||||
DEFAULT_METRICS_REPORTER_TYPE.name());
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_HOST), GRAPHITE_SERVER_HOST,
|
||||
DEFAULT_GRAPHITE_SERVER_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(GRAPHITE_SERVER_PORT), GRAPHITE_SERVER_PORT,
|
||||
String.valueOf(DEFAULT_GRAPHITE_SERVER_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(JMX_HOST), JMX_HOST,
|
||||
DEFAULT_JMX_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(JMX_PORT), JMX_PORT,
|
||||
String.valueOf(DEFAULT_JMX_PORT));
|
||||
MetricsReporterType reporterType = MetricsReporterType.valueOf(props.getProperty(METRICS_REPORTER_TYPE));
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.DATADOG,
|
||||
HoodieMetricsDatadogConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, !props.containsKey(METRICS_REPORTER_CLASS),
|
||||
METRICS_REPORTER_CLASS, DEFAULT_METRICS_REPORTER_CLASS);
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS_PUSHGATEWAY,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
|
||||
setDefaultOnCondition(props, reporterType == MetricsReporterType.PROMETHEUS,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(props).build());
|
||||
|
||||
return config;
|
||||
hoodieMetricsConfig.setDefaults(HoodieMetricsConfig.class.getName());
|
||||
|
||||
MetricsReporterType reporterType = MetricsReporterType.valueOf(hoodieMetricsConfig.getString(METRICS_REPORTER_TYPE));
|
||||
|
||||
hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.DATADOG,
|
||||
HoodieMetricsDatadogConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
|
||||
hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.PROMETHEUS_PUSHGATEWAY,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
|
||||
hoodieMetricsConfig.setDefaultOnCondition(reporterType == MetricsReporterType.PROMETHEUS,
|
||||
HoodieMetricsPrometheusConfig.newBuilder().fromProperties(hoodieMetricsConfig.getProps()).build());
|
||||
return hoodieMetricsConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
@@ -32,24 +33,69 @@ import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
|
||||
* {@link org.apache.hudi.metrics.MetricsReporterType#DATADOG}
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieMetricsDatadogConfig extends DefaultHoodieConfig {
|
||||
public class HoodieMetricsDatadogConfig extends HoodieConfig {
|
||||
|
||||
public static final String DATADOG_PREFIX = METRIC_PREFIX + ".datadog";
|
||||
public static final String DATADOG_REPORT_PERIOD_SECONDS = DATADOG_PREFIX + ".report.period.seconds";
|
||||
public static final int DEFAULT_DATADOG_REPORT_PERIOD_SECONDS = 30;
|
||||
public static final String DATADOG_API_SITE = DATADOG_PREFIX + ".api.site";
|
||||
public static final String DATADOG_API_KEY = DATADOG_PREFIX + ".api.key";
|
||||
public static final String DATADOG_API_KEY_SKIP_VALIDATION = DATADOG_PREFIX + ".api.key.skip.validation";
|
||||
public static final boolean DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION = false;
|
||||
public static final String DATADOG_API_KEY_SUPPLIER = DATADOG_PREFIX + ".api.key.supplier";
|
||||
public static final String DATADOG_API_TIMEOUT_SECONDS = DATADOG_PREFIX + ".api.timeout.seconds";
|
||||
public static final int DEFAULT_DATADOG_API_TIMEOUT_SECONDS = 3;
|
||||
public static final String DATADOG_METRIC_PREFIX = DATADOG_PREFIX + ".metric.prefix";
|
||||
public static final String DATADOG_METRIC_HOST = DATADOG_PREFIX + ".metric.host";
|
||||
public static final String DATADOG_METRIC_TAGS = DATADOG_PREFIX + ".metric.tags";
|
||||
|
||||
private HoodieMetricsDatadogConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<Integer> DATADOG_REPORT_PERIOD_SECONDS = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".report.period.seconds")
|
||||
.defaultValue(30)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog report period in seconds. Default to 30.");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_API_SITE = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".api.site")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog API site: EU or US");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_API_KEY = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".api.key")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog API key");
|
||||
|
||||
public static final ConfigProperty<Boolean> DATADOG_API_KEY_SKIP_VALIDATION = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".api.key.skip.validation")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Before sending metrics via Datadog API, whether to skip validating Datadog API key or not. "
|
||||
+ "Default to false.");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_API_KEY_SUPPLIER = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".api.key.supplier")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog API key supplier to supply the API key at runtime. "
|
||||
+ "This will take effect if hoodie.metrics.datadog.api.key is not set.");
|
||||
|
||||
public static final ConfigProperty<Integer> DATADOG_API_TIMEOUT_SECONDS = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".api.timeout.seconds")
|
||||
.defaultValue(3)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog API timeout in seconds. Default to 3.");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_METRIC_PREFIX = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".metric.prefix")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog metric prefix to be prepended to each metric name with a dot as delimiter. "
|
||||
+ "For example, if it is set to foo, foo. will be prepended.");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_METRIC_HOST = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".metric.host")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog metric host to be sent along with metrics data.");
|
||||
|
||||
public static final ConfigProperty<String> DATADOG_METRIC_TAGS = ConfigProperty
|
||||
.key(DATADOG_PREFIX + ".metric.tags")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Datadog metric tags (comma-delimited) to be sent along with metrics data.");
|
||||
|
||||
private HoodieMetricsDatadogConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieMetricsDatadogConfig.Builder newBuilder() {
|
||||
@@ -58,70 +104,61 @@ public class HoodieMetricsDatadogConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieMetricsDatadogConfig metricsDatadogConfig = new HoodieMetricsDatadogConfig();
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.metricsDatadogConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogReportPeriodSeconds(int period) {
|
||||
props.setProperty(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
|
||||
metricsDatadogConfig.setValue(DATADOG_REPORT_PERIOD_SECONDS, String.valueOf(period));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiSite(String apiSite) {
|
||||
props.setProperty(DATADOG_API_SITE, apiSite);
|
||||
metricsDatadogConfig.setValue(DATADOG_API_SITE, apiSite);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKey(String apiKey) {
|
||||
props.setProperty(DATADOG_API_KEY, apiKey);
|
||||
metricsDatadogConfig.setValue(DATADOG_API_KEY, apiKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKeySkipValidation(boolean skip) {
|
||||
props.setProperty(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
|
||||
metricsDatadogConfig.setValue(DATADOG_API_KEY_SKIP_VALIDATION, String.valueOf(skip));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiKeySupplier(String apiKeySupplier) {
|
||||
props.setProperty(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
|
||||
metricsDatadogConfig.setValue(DATADOG_API_KEY_SUPPLIER, apiKeySupplier);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogApiTimeoutSeconds(int timeout) {
|
||||
props.setProperty(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
|
||||
metricsDatadogConfig.setValue(DATADOG_API_TIMEOUT_SECONDS, String.valueOf(timeout));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogPrefix(String prefix) {
|
||||
props.setProperty(DATADOG_METRIC_PREFIX, prefix);
|
||||
metricsDatadogConfig.setValue(DATADOG_METRIC_PREFIX, prefix);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogHost(String host) {
|
||||
props.setProperty(DATADOG_METRIC_HOST, host);
|
||||
metricsDatadogConfig.setValue(DATADOG_METRIC_HOST, host);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDatadogTags(String tags) {
|
||||
props.setProperty(DATADOG_METRIC_TAGS, tags);
|
||||
metricsDatadogConfig.setValue(DATADOG_METRIC_TAGS, tags);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsDatadogConfig build() {
|
||||
HoodieMetricsDatadogConfig config = new HoodieMetricsDatadogConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_REPORT_PERIOD_SECONDS),
|
||||
DATADOG_REPORT_PERIOD_SECONDS,
|
||||
String.valueOf(DEFAULT_DATADOG_REPORT_PERIOD_SECONDS));
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_API_KEY_SKIP_VALIDATION),
|
||||
DATADOG_API_KEY_SKIP_VALIDATION,
|
||||
String.valueOf(DEFAULT_DATADOG_API_KEY_SKIP_VALIDATION));
|
||||
setDefaultOnCondition(props, !props.containsKey(DATADOG_API_TIMEOUT_SECONDS),
|
||||
DATADOG_API_TIMEOUT_SECONDS,
|
||||
String.valueOf(DEFAULT_DATADOG_API_TIMEOUT_SECONDS));
|
||||
return config;
|
||||
metricsDatadogConfig.setDefaults(HoodieMetricsDatadogConfig.class.getName());
|
||||
return metricsDatadogConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,43 +18,65 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMetricsConfig.METRIC_PREFIX;
|
||||
|
||||
public class HoodieMetricsPrometheusConfig extends DefaultHoodieConfig {
|
||||
public class HoodieMetricsPrometheusConfig extends HoodieConfig {
|
||||
|
||||
// Prometheus PushGateWay
|
||||
public static final String PUSHGATEWAY_PREFIX = METRIC_PREFIX + ".pushgateway";
|
||||
|
||||
public static final String PUSHGATEWAY_HOST = PUSHGATEWAY_PREFIX + ".host";
|
||||
public static final String DEFAULT_PUSHGATEWAY_HOST = "localhost";
|
||||
public static final ConfigProperty<String> PUSHGATEWAY_HOST = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".host")
|
||||
.defaultValue("localhost")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String PUSHGATEWAY_PORT = PUSHGATEWAY_PREFIX + ".port";
|
||||
public static final int DEFAULT_PUSHGATEWAY_PORT = 9091;
|
||||
public static final ConfigProperty<Integer> PUSHGATEWAY_PORT = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".port")
|
||||
.defaultValue(9091)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String PUSHGATEWAY_REPORT_PERIOD_SECONDS = PUSHGATEWAY_PREFIX + ".report.period.seconds";
|
||||
public static final int DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS = 30;
|
||||
public static final ConfigProperty<Integer> PUSHGATEWAY_REPORT_PERIOD_SECONDS = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".report.period.seconds")
|
||||
.defaultValue(30)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String PUSHGATEWAY_DELETE_ON_SHUTDOWN = PUSHGATEWAY_PREFIX + ".delete.on.shutdown";
|
||||
public static final boolean DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN = true;
|
||||
public static final ConfigProperty<Boolean> PUSHGATEWAY_DELETE_ON_SHUTDOWN = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".delete.on.shutdown")
|
||||
.defaultValue(true)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final String PUSHGATEWAY_JOB_NAME = PUSHGATEWAY_PREFIX + ".job.name";
|
||||
public static final String DEFAULT_PUSHGATEWAY_JOB_NAME = "";
|
||||
|
||||
public static final String PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = PUSHGATEWAY_PREFIX + ".random.job.name.suffix";
|
||||
public static final boolean DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = true;
|
||||
public static final ConfigProperty<String> PUSHGATEWAY_JOB_NAME = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".job.name")
|
||||
.defaultValue("")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<Boolean> PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX = ConfigProperty
|
||||
.key(PUSHGATEWAY_PREFIX + ".random.job.name.suffix")
|
||||
.defaultValue(true)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
// Prometheus HttpServer
|
||||
public static final String PROMETHEUS_PREFIX = METRIC_PREFIX + ".prometheus";
|
||||
public static final String PROMETHEUS_PORT = PROMETHEUS_PREFIX + ".port";
|
||||
public static final int DEFAULT_PROMETHEUS_PORT = 9090;
|
||||
|
||||
public HoodieMetricsPrometheusConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<Integer> PROMETHEUS_PORT = ConfigProperty
|
||||
.key(PROMETHEUS_PREFIX + ".port")
|
||||
.defaultValue(9090)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
|
||||
private HoodieMetricsPrometheusConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieMetricsPrometheusConfig.Builder newBuilder() {
|
||||
@@ -68,35 +90,16 @@ public class HoodieMetricsPrometheusConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private Properties props = new Properties();
|
||||
private HoodieMetricsPrometheusConfig hoodieMetricsPrometheusConfig = new HoodieMetricsPrometheusConfig();
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.hoodieMetricsPrometheusConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieMetricsPrometheusConfig build() {
|
||||
HoodieMetricsPrometheusConfig config = new HoodieMetricsPrometheusConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PROMETHEUS_PORT), PROMETHEUS_PORT,
|
||||
String.valueOf(DEFAULT_PROMETHEUS_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_HOST),
|
||||
PUSHGATEWAY_HOST,
|
||||
DEFAULT_PUSHGATEWAY_HOST);
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_PORT),
|
||||
PUSHGATEWAY_PORT,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_PORT));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_REPORT_PERIOD_SECONDS),
|
||||
PUSHGATEWAY_REPORT_PERIOD_SECONDS,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_REPORT_PERIOD_SECONDS));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_DELETE_ON_SHUTDOWN),
|
||||
PUSHGATEWAY_DELETE_ON_SHUTDOWN,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_DELETE_ON_SHUTDOWN));
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_JOB_NAME),
|
||||
PUSHGATEWAY_JOB_NAME, DEFAULT_PUSHGATEWAY_JOB_NAME);
|
||||
setDefaultOnCondition(props, !props.containsKey(PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX),
|
||||
PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX,
|
||||
String.valueOf(DEFAULT_PUSHGATEWAY_RANDOM_JOB_NAME_SUFFIX));
|
||||
return config;
|
||||
hoodieMetricsPrometheusConfig.setDefaults(HoodieMetricsPrometheusConfig.class.getName());
|
||||
return hoodieMetricsPrometheusConfig;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -18,25 +18,34 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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;
|
||||
import java.io.IOException;
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL;
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.DEFAULT_PAYLOAD_ORDERING_FIELD_VAL;
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP;
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP;
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_EVENT_TIME_FIELD_PROP_KEY;
|
||||
import static org.apache.hudi.common.model.HoodiePayloadProps.PAYLOAD_ORDERING_FIELD_PROP_KEY;
|
||||
|
||||
/**
|
||||
* Hoodie payload related configs.
|
||||
*/
|
||||
public class HoodiePayloadConfig extends DefaultHoodieConfig {
|
||||
public class HoodiePayloadConfig extends HoodieConfig {
|
||||
|
||||
public HoodiePayloadConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> PAYLOAD_ORDERING_FIELD_PROP = ConfigProperty
|
||||
.key(PAYLOAD_ORDERING_FIELD_PROP_KEY)
|
||||
.defaultValue("ts")
|
||||
.withDocumentation("Property to hold the payload ordering field name");
|
||||
|
||||
public static final ConfigProperty<String> PAYLOAD_EVENT_TIME_FIELD_PROP = ConfigProperty
|
||||
.key(PAYLOAD_EVENT_TIME_FIELD_PROP_KEY)
|
||||
.defaultValue("ts")
|
||||
.withDocumentation("Property for payload event time field");
|
||||
|
||||
private HoodiePayloadConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodiePayloadConfig.Builder newBuilder() {
|
||||
@@ -45,37 +54,33 @@ public class HoodiePayloadConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodiePayloadConfig payloadConfig = new HoodiePayloadConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.payloadConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.payloadConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPayloadOrderingField(String payloadOrderingField) {
|
||||
props.setProperty(PAYLOAD_ORDERING_FIELD_PROP, String.valueOf(payloadOrderingField));
|
||||
payloadConfig.setValue(PAYLOAD_ORDERING_FIELD_PROP, String.valueOf(payloadOrderingField));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withPayloadEventTimeField(String payloadEventTimeField) {
|
||||
props.setProperty(PAYLOAD_EVENT_TIME_FIELD_PROP, String.valueOf(payloadEventTimeField));
|
||||
payloadConfig.setValue(PAYLOAD_EVENT_TIME_FIELD_PROP, String.valueOf(payloadEventTimeField));
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodiePayloadConfig build() {
|
||||
HoodiePayloadConfig config = new HoodiePayloadConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_ORDERING_FIELD_PROP), PAYLOAD_ORDERING_FIELD_PROP,
|
||||
String.valueOf(DEFAULT_PAYLOAD_ORDERING_FIELD_VAL));
|
||||
setDefaultOnCondition(props, !props.containsKey(PAYLOAD_EVENT_TIME_FIELD_PROP), PAYLOAD_EVENT_TIME_FIELD_PROP,
|
||||
String.valueOf(DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL));
|
||||
return config;
|
||||
payloadConfig.setDefaults(HoodiePayloadConfig.class.getName());
|
||||
return payloadConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -18,7 +18,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
import org.apache.hudi.common.config.DefaultHoodieConfig;
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
@@ -31,51 +32,97 @@ import java.util.Properties;
|
||||
* Storage related config.
|
||||
*/
|
||||
@Immutable
|
||||
public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||
public class HoodieStorageConfig extends HoodieConfig {
|
||||
|
||||
public static final String PARQUET_FILE_MAX_BYTES = "hoodie.parquet.max.file.size";
|
||||
public static final String DEFAULT_PARQUET_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||
public static final String PARQUET_BLOCK_SIZE_BYTES = "hoodie.parquet.block.size";
|
||||
public static final String DEFAULT_PARQUET_BLOCK_SIZE_BYTES = DEFAULT_PARQUET_FILE_MAX_BYTES;
|
||||
public static final String PARQUET_PAGE_SIZE_BYTES = "hoodie.parquet.page.size";
|
||||
public static final String DEFAULT_PARQUET_PAGE_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
public static final ConfigProperty<String> PARQUET_FILE_MAX_BYTES = ConfigProperty
|
||||
.key("hoodie.parquet.max.file.size")
|
||||
.defaultValue(String.valueOf(120 * 1024 * 1024))
|
||||
.withDocumentation("Target size for parquet files produced by Hudi write phases. "
|
||||
+ "For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.");
|
||||
|
||||
public static final String HFILE_FILE_MAX_BYTES = "hoodie.hfile.max.file.size";
|
||||
public static final String HFILE_BLOCK_SIZE_BYTES = "hoodie.hfile.block.size";
|
||||
public static final String DEFAULT_HFILE_BLOCK_SIZE_BYTES = String.valueOf(1 * 1024 * 1024);
|
||||
public static final String DEFAULT_HFILE_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||
public static final ConfigProperty<String> PARQUET_BLOCK_SIZE_BYTES = ConfigProperty
|
||||
.key("hoodie.parquet.block.size")
|
||||
.defaultValue(String.valueOf(120 * 1024 * 1024))
|
||||
.withDocumentation("Parquet RowGroup size. Its better this is same as the file size, so that a single column "
|
||||
+ "within a file is stored continuously on disk");
|
||||
|
||||
public static final String ORC_FILE_MAX_BYTES = "hoodie.orc.max.file.size";
|
||||
public static final String DEFAULT_ORC_FILE_MAX_BYTES = String.valueOf(120 * 1024 * 1024);
|
||||
// size of the memory buffer in bytes for writing
|
||||
public static final String ORC_STRIPE_SIZE = "hoodie.orc.stripe.size";
|
||||
public static final String DEFAULT_ORC_STRIPE_SIZE = String.valueOf(64 * 1024 * 1024);
|
||||
// file system block size
|
||||
public static final String ORC_BLOCK_SIZE = "hoodie.orc.block.size";
|
||||
public static final String DEFAULT_ORC_BLOCK_SIZE = DEFAULT_ORC_FILE_MAX_BYTES;
|
||||
public static final ConfigProperty<String> PARQUET_PAGE_SIZE_BYTES = ConfigProperty
|
||||
.key("hoodie.parquet.page.size")
|
||||
.defaultValue(String.valueOf(1 * 1024 * 1024))
|
||||
.withDocumentation("Parquet page size. Page is the unit of read within a parquet file. "
|
||||
+ "Within a block, pages are compressed seperately.");
|
||||
|
||||
public static final ConfigProperty<String> ORC_FILE_MAX_BYTES = ConfigProperty
|
||||
.key("hoodie.orc.max.file.size")
|
||||
.defaultValue(String.valueOf(120 * 1024 * 1024))
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> ORC_STRIPE_SIZE = ConfigProperty
|
||||
.key("hoodie.orc.stripe.size")
|
||||
.defaultValue(String.valueOf(64 * 1024 * 1024))
|
||||
.withDocumentation("Size of the memory buffer in bytes for writing");
|
||||
|
||||
public static final ConfigProperty<String> ORC_BLOCK_SIZE = ConfigProperty
|
||||
.key("hoodie.orc.block.size")
|
||||
.defaultValue(ORC_FILE_MAX_BYTES.defaultValue())
|
||||
.withDocumentation("File system block size");
|
||||
|
||||
public static final ConfigProperty<String> HFILE_FILE_MAX_BYTES = ConfigProperty
|
||||
.key("hoodie.hfile.max.file.size")
|
||||
.defaultValue(String.valueOf(120 * 1024 * 1024))
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> HFILE_BLOCK_SIZE_BYTES = ConfigProperty
|
||||
.key("hoodie.hfile.block.size")
|
||||
.defaultValue(String.valueOf(1 * 1024 * 1024))
|
||||
.withDocumentation("");
|
||||
|
||||
// used to size log files
|
||||
public static final String LOGFILE_SIZE_MAX_BYTES = "hoodie.logfile.max.size";
|
||||
public static final String DEFAULT_LOGFILE_SIZE_MAX_BYTES = String.valueOf(1024 * 1024 * 1024); // 1 GB
|
||||
// used to size data blocks in log file
|
||||
public static final String LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = "hoodie.logfile.data.block.max.size";
|
||||
public static final String DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = String.valueOf(256 * 1024 * 1024); // 256 MB
|
||||
public static final String PARQUET_COMPRESSION_RATIO = "hoodie.parquet.compression.ratio";
|
||||
// Default compression ratio for parquet
|
||||
public static final String DEFAULT_STREAM_COMPRESSION_RATIO = String.valueOf(0.1);
|
||||
public static final String PARQUET_COMPRESSION_CODEC = "hoodie.parquet.compression.codec";
|
||||
public static final String HFILE_COMPRESSION_ALGORITHM = "hoodie.hfile.compression.algorithm";
|
||||
public static final String ORC_COMPRESSION_CODEC = "hoodie.orc.compression.codec";
|
||||
// Default compression codec for parquet
|
||||
public static final String DEFAULT_PARQUET_COMPRESSION_CODEC = "gzip";
|
||||
public static final String DEFAULT_HFILE_COMPRESSION_ALGORITHM = "GZ";
|
||||
public static final String DEFAULT_ORC_COMPRESSION_CODEC = "ZLIB";
|
||||
public static final String LOGFILE_TO_PARQUET_COMPRESSION_RATIO = "hoodie.logfile.to.parquet.compression.ratio";
|
||||
// Default compression ratio for log file to parquet, general 3x
|
||||
public static final String DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO = String.valueOf(0.35);
|
||||
public static final ConfigProperty<String> LOGFILE_SIZE_MAX_BYTES = ConfigProperty
|
||||
.key("hoodie.logfile.max.size")
|
||||
.defaultValue(String.valueOf(1024 * 1024 * 1024)) // 1 GB
|
||||
.withDocumentation("LogFile max size. This is the maximum size allowed for a log file "
|
||||
+ "before it is rolled over to the next version.");
|
||||
|
||||
private HoodieStorageConfig(Properties props) {
|
||||
super(props);
|
||||
// used to size data blocks in log file
|
||||
public static final ConfigProperty<String> LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES = ConfigProperty
|
||||
.key("hoodie.logfile.data.block.max.size")
|
||||
.defaultValue(String.valueOf(256 * 1024 * 1024))
|
||||
.withDocumentation("LogFile Data block max size. This is the maximum size allowed for a single data block "
|
||||
+ "to be appended to a log file. This helps to make sure the data appended to the log file is broken up "
|
||||
+ "into sizable blocks to prevent from OOM errors. This size should be greater than the JVM memory.");
|
||||
|
||||
public static final ConfigProperty<String> PARQUET_COMPRESSION_RATIO = ConfigProperty
|
||||
.key("hoodie.parquet.compression.ratio")
|
||||
.defaultValue(String.valueOf(0.1))
|
||||
.withDocumentation("Expected compression of parquet data used by Hudi, when it tries to size new parquet files. "
|
||||
+ "Increase this value, if bulk_insert is producing smaller than expected sized files");
|
||||
|
||||
// Default compression codec for parquet
|
||||
public static final ConfigProperty<String> PARQUET_COMPRESSION_CODEC = ConfigProperty
|
||||
.key("hoodie.parquet.compression.codec")
|
||||
.defaultValue("gzip")
|
||||
.withDocumentation("Compression Codec for parquet files");
|
||||
|
||||
public static final ConfigProperty<String> HFILE_COMPRESSION_ALGORITHM = ConfigProperty
|
||||
.key("hoodie.hfile.compression.algorithm")
|
||||
.defaultValue("GZ")
|
||||
.withDocumentation("");
|
||||
|
||||
public static final ConfigProperty<String> ORC_COMPRESSION_CODEC = ConfigProperty
|
||||
.key("hoodie.orc.compression.codec")
|
||||
.defaultValue("ZLIB")
|
||||
.withDocumentation("");
|
||||
|
||||
// Default compression ratio for log file to parquet, general 3x
|
||||
public static final ConfigProperty<String> LOGFILE_TO_PARQUET_COMPRESSION_RATIO = ConfigProperty
|
||||
.key("hoodie.logfile.to.parquet.compression.ratio")
|
||||
.defaultValue(String.valueOf(0.35))
|
||||
.withDocumentation("Expected additional compression as records move from log files to parquet. Used for merge_on_read "
|
||||
+ "table to send inserts into log files & control the size of compacted parquet file.");
|
||||
|
||||
private HoodieStorageConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieStorageConfig.Builder newBuilder() {
|
||||
@@ -84,131 +131,98 @@ public class HoodieStorageConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieStorageConfig storageConfig = new HoodieStorageConfig();
|
||||
|
||||
public Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.storageConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.storageConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetMaxFileSize(long maxFileSize) {
|
||||
props.setProperty(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
storageConfig.setValue(PARQUET_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetBlockSize(int blockSize) {
|
||||
props.setProperty(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
storageConfig.setValue(PARQUET_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetPageSize(int pageSize) {
|
||||
props.setProperty(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
||||
storageConfig.setValue(PARQUET_PAGE_SIZE_BYTES, String.valueOf(pageSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hfileMaxFileSize(long maxFileSize) {
|
||||
props.setProperty(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
storageConfig.setValue(HFILE_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hfileBlockSize(int blockSize) {
|
||||
props.setProperty(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
storageConfig.setValue(HFILE_BLOCK_SIZE_BYTES, String.valueOf(blockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileDataBlockMaxSize(int dataBlockSize) {
|
||||
props.setProperty(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
|
||||
storageConfig.setValue(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, String.valueOf(dataBlockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileMaxSize(int logFileSize) {
|
||||
props.setProperty(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
|
||||
storageConfig.setValue(LOGFILE_SIZE_MAX_BYTES, String.valueOf(logFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionRatio(double parquetCompressionRatio) {
|
||||
props.setProperty(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
|
||||
storageConfig.setValue(PARQUET_COMPRESSION_RATIO, String.valueOf(parquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder parquetCompressionCodec(String parquetCompressionCodec) {
|
||||
props.setProperty(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
|
||||
storageConfig.setValue(PARQUET_COMPRESSION_CODEC, parquetCompressionCodec);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder hfileCompressionAlgorithm(String hfileCompressionAlgorithm) {
|
||||
props.setProperty(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
|
||||
storageConfig.setValue(HFILE_COMPRESSION_ALGORITHM, hfileCompressionAlgorithm);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder logFileToParquetCompressionRatio(double logFileToParquetCompressionRatio) {
|
||||
props.setProperty(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
|
||||
storageConfig.setValue(LOGFILE_TO_PARQUET_COMPRESSION_RATIO, String.valueOf(logFileToParquetCompressionRatio));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder orcMaxFileSize(long maxFileSize) {
|
||||
props.setProperty(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
storageConfig.setValue(ORC_FILE_MAX_BYTES, String.valueOf(maxFileSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder orcStripeSize(int orcStripeSize) {
|
||||
props.setProperty(ORC_STRIPE_SIZE, String.valueOf(orcStripeSize));
|
||||
storageConfig.setValue(ORC_STRIPE_SIZE, String.valueOf(orcStripeSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder orcBlockSize(int orcBlockSize) {
|
||||
props.setProperty(ORC_BLOCK_SIZE, String.valueOf(orcBlockSize));
|
||||
storageConfig.setValue(ORC_BLOCK_SIZE, String.valueOf(orcBlockSize));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder orcCompressionCodec(String orcCompressionCodec) {
|
||||
props.setProperty(ORC_COMPRESSION_CODEC, orcCompressionCodec);
|
||||
storageConfig.setValue(ORC_COMPRESSION_CODEC, orcCompressionCodec);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieStorageConfig build() {
|
||||
HoodieStorageConfig config = new HoodieStorageConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_FILE_MAX_BYTES), PARQUET_FILE_MAX_BYTES,
|
||||
DEFAULT_PARQUET_FILE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_BLOCK_SIZE_BYTES), PARQUET_BLOCK_SIZE_BYTES,
|
||||
DEFAULT_PARQUET_BLOCK_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_PAGE_SIZE_BYTES), PARQUET_PAGE_SIZE_BYTES,
|
||||
DEFAULT_PARQUET_PAGE_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES),
|
||||
LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES, DEFAULT_LOGFILE_DATA_BLOCK_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_SIZE_MAX_BYTES), LOGFILE_SIZE_MAX_BYTES,
|
||||
DEFAULT_LOGFILE_SIZE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_RATIO), PARQUET_COMPRESSION_RATIO,
|
||||
DEFAULT_STREAM_COMPRESSION_RATIO);
|
||||
setDefaultOnCondition(props, !props.containsKey(PARQUET_COMPRESSION_CODEC), PARQUET_COMPRESSION_CODEC,
|
||||
DEFAULT_PARQUET_COMPRESSION_CODEC);
|
||||
setDefaultOnCondition(props, !props.containsKey(LOGFILE_TO_PARQUET_COMPRESSION_RATIO),
|
||||
LOGFILE_TO_PARQUET_COMPRESSION_RATIO, DEFAULT_LOGFILE_TO_PARQUET_COMPRESSION_RATIO);
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_BLOCK_SIZE_BYTES), HFILE_BLOCK_SIZE_BYTES,
|
||||
DEFAULT_HFILE_BLOCK_SIZE_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_COMPRESSION_ALGORITHM), HFILE_COMPRESSION_ALGORITHM,
|
||||
DEFAULT_HFILE_COMPRESSION_ALGORITHM);
|
||||
setDefaultOnCondition(props, !props.containsKey(HFILE_FILE_MAX_BYTES), HFILE_FILE_MAX_BYTES,
|
||||
DEFAULT_HFILE_FILE_MAX_BYTES);
|
||||
|
||||
setDefaultOnCondition(props, !props.containsKey(ORC_FILE_MAX_BYTES), ORC_FILE_MAX_BYTES,
|
||||
DEFAULT_ORC_FILE_MAX_BYTES);
|
||||
setDefaultOnCondition(props, !props.containsKey(ORC_STRIPE_SIZE), ORC_STRIPE_SIZE,
|
||||
DEFAULT_ORC_STRIPE_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(ORC_BLOCK_SIZE), ORC_BLOCK_SIZE,
|
||||
DEFAULT_ORC_BLOCK_SIZE);
|
||||
setDefaultOnCondition(props, !props.containsKey(ORC_COMPRESSION_CODEC), ORC_COMPRESSION_CODEC,
|
||||
DEFAULT_ORC_COMPRESSION_CODEC);
|
||||
|
||||
return config;
|
||||
storageConfig.setDefaults(HoodieStorageConfig.class.getName());
|
||||
return storageConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@@ -17,7 +17,8 @@
|
||||
|
||||
package org.apache.hudi.config;
|
||||
|
||||
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;
|
||||
@@ -27,24 +28,44 @@ import java.util.Properties;
|
||||
/**
|
||||
* Write callback related config.
|
||||
*/
|
||||
public class HoodieWriteCommitCallbackConfig extends DefaultHoodieConfig {
|
||||
public class HoodieWriteCommitCallbackConfig extends HoodieConfig {
|
||||
|
||||
public static final String CALLBACK_PREFIX = "hoodie.write.commit.callback.";
|
||||
public static final String CALLBACK_ON = CALLBACK_PREFIX + "on";
|
||||
public static final boolean DEFAULT_CALLBACK_ON = false;
|
||||
|
||||
public static final String CALLBACK_CLASS_PROP = CALLBACK_PREFIX + "class";
|
||||
public static final String DEFAULT_CALLBACK_CLASS_PROP = "org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback";
|
||||
public static final ConfigProperty<Boolean> CALLBACK_ON = ConfigProperty
|
||||
.key(CALLBACK_PREFIX + "on")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Turn callback on/off. off by default.");
|
||||
|
||||
public static final ConfigProperty<String> CALLBACK_CLASS_PROP = ConfigProperty
|
||||
.key(CALLBACK_PREFIX + "class")
|
||||
.defaultValue("org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Full path of callback class and must be a subclass of HoodieWriteCommitCallback class, "
|
||||
+ "org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback by default");
|
||||
|
||||
// ***** HTTP callback configs *****
|
||||
public static final String CALLBACK_HTTP_URL_PROP = CALLBACK_PREFIX + "http.url";
|
||||
public static final String CALLBACK_HTTP_API_KEY = CALLBACK_PREFIX + "http.api.key";
|
||||
public static final String DEFAULT_CALLBACK_HTTP_API_KEY = "hudi_write_commit_http_callback";
|
||||
public static final String CALLBACK_HTTP_TIMEOUT_SECONDS = CALLBACK_PREFIX + "http.timeout.seconds";
|
||||
public static final int DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS = 3;
|
||||
public static final ConfigProperty<String> CALLBACK_HTTP_URL_PROP = ConfigProperty
|
||||
.key(CALLBACK_PREFIX + "http.url")
|
||||
.noDefaultValue()
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Callback host to be sent along with callback messages");
|
||||
|
||||
private HoodieWriteCommitCallbackConfig(Properties props) {
|
||||
super(props);
|
||||
public static final ConfigProperty<String> CALLBACK_HTTP_API_KEY = ConfigProperty
|
||||
.key(CALLBACK_PREFIX + "http.api.key")
|
||||
.defaultValue("hudi_write_commit_http_callback")
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Http callback API key. hudi_write_commit_http_callback by default");
|
||||
|
||||
public static final ConfigProperty<Integer> CALLBACK_HTTP_TIMEOUT_SECONDS = ConfigProperty
|
||||
.key(CALLBACK_PREFIX + "http.timeout.seconds")
|
||||
.defaultValue(3)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Callback timeout in seconds. 3 by default");
|
||||
|
||||
private HoodieWriteCommitCallbackConfig() {
|
||||
super();
|
||||
}
|
||||
|
||||
public static HoodieWriteCommitCallbackConfig.Builder newBuilder() {
|
||||
@@ -53,54 +74,48 @@ public class HoodieWriteCommitCallbackConfig extends DefaultHoodieConfig {
|
||||
|
||||
public static class Builder {
|
||||
|
||||
private final Properties props = new Properties();
|
||||
private final HoodieWriteCommitCallbackConfig writeCommitCallbackConfig = new HoodieWriteCommitCallbackConfig();
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder fromFile(File propertiesFile) throws IOException {
|
||||
try (FileReader reader = new FileReader(propertiesFile)) {
|
||||
this.props.load(reader);
|
||||
this.writeCommitCallbackConfig.getProps().load(reader);
|
||||
return this;
|
||||
}
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder fromProperties(Properties props) {
|
||||
this.props.putAll(props);
|
||||
this.writeCommitCallbackConfig.getProps().putAll(props);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder writeCommitCallbackOn(String callbackOn) {
|
||||
props.setProperty(CALLBACK_ON, callbackOn);
|
||||
writeCommitCallbackConfig.setValue(CALLBACK_ON, callbackOn);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder withCallbackClass(String callbackClass) {
|
||||
props.setProperty(CALLBACK_CLASS_PROP, callbackClass);
|
||||
writeCommitCallbackConfig.setValue(CALLBACK_CLASS_PROP, callbackClass);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig.Builder withCallbackHttpUrl(String url) {
|
||||
props.setProperty(CALLBACK_HTTP_URL_PROP, url);
|
||||
writeCommitCallbackConfig.setValue(CALLBACK_HTTP_URL_PROP, url);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCallbackHttpTimeoutSeconds(String timeoutSeconds) {
|
||||
props.setProperty(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
|
||||
writeCommitCallbackConfig.setValue(CALLBACK_HTTP_TIMEOUT_SECONDS, timeoutSeconds);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withCallbackHttpApiKey(String apiKey) {
|
||||
props.setProperty(CALLBACK_HTTP_API_KEY, apiKey);
|
||||
writeCommitCallbackConfig.setValue(CALLBACK_HTTP_API_KEY, apiKey);
|
||||
return this;
|
||||
}
|
||||
|
||||
public HoodieWriteCommitCallbackConfig build() {
|
||||
HoodieWriteCommitCallbackConfig config = new HoodieWriteCommitCallbackConfig(props);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_ON), CALLBACK_ON, String.valueOf(DEFAULT_CALLBACK_ON));
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_CLASS_PROP), CALLBACK_CLASS_PROP, DEFAULT_CALLBACK_CLASS_PROP);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_API_KEY), CALLBACK_HTTP_API_KEY, DEFAULT_CALLBACK_HTTP_API_KEY);
|
||||
setDefaultOnCondition(props, !props.containsKey(CALLBACK_HTTP_TIMEOUT_SECONDS), CALLBACK_HTTP_TIMEOUT_SECONDS,
|
||||
String.valueOf(DEFAULT_CALLBACK_HTTP_TIMEOUT_SECONDS));
|
||||
|
||||
return config;
|
||||
writeCommitCallbackConfig.setDefaults(HoodieWriteCommitCallbackConfig.class.getName());
|
||||
return writeCommitCallbackConfig;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
File diff suppressed because it is too large
Load Diff
@@ -196,7 +196,7 @@ public class HoodieMergeHandle<T extends HoodieRecordPayload, I, K, O> extends H
|
||||
protected void initializeIncomingRecordsMap() {
|
||||
try {
|
||||
// Load the new records in a map
|
||||
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config.getProps());
|
||||
long memoryForMerge = IOUtils.getMaxMemoryPerPartitionMerge(taskContextSupplier, config);
|
||||
LOG.info("MaxMemoryPerPartitionMerge => " + memoryForMerge);
|
||||
this.keyToNewRecords = new ExternalSpillableMap<>(memoryForMerge, config.getSpillableMapBasePath(),
|
||||
new DefaultSizeEstimator(), new HoodieRecordSizeEstimator(tableSchema));
|
||||
|
||||
@@ -18,15 +18,12 @@
|
||||
|
||||
package org.apache.hudi.io;
|
||||
|
||||
import org.apache.hudi.common.config.HoodieConfig;
|
||||
import org.apache.hudi.common.engine.EngineProperty;
|
||||
import org.apache.hudi.common.engine.TaskContextSupplier;
|
||||
import org.apache.hudi.common.util.Option;
|
||||
|
||||
import java.util.Properties;
|
||||
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.DEFAULT_MIN_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_COMPACTION_PROP;
|
||||
import static org.apache.hudi.config.HoodieMemoryConfig.MAX_MEMORY_FOR_MERGE_PROP;
|
||||
@@ -56,19 +53,19 @@ public class IOUtils {
|
||||
}
|
||||
}
|
||||
|
||||
public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, Properties properties) {
|
||||
if (properties.containsKey(MAX_MEMORY_FOR_MERGE_PROP)) {
|
||||
return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_MERGE_PROP));
|
||||
public static long getMaxMemoryPerPartitionMerge(TaskContextSupplier context, HoodieConfig hoodieConfig) {
|
||||
if (hoodieConfig.contains(MAX_MEMORY_FOR_MERGE_PROP)) {
|
||||
return hoodieConfig.getLong(MAX_MEMORY_FOR_MERGE_PROP);
|
||||
}
|
||||
String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_MERGE_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_MERGE);
|
||||
String fraction = hoodieConfig.getStringOrDefault(MAX_MEMORY_FRACTION_FOR_MERGE_PROP);
|
||||
return getMaxMemoryAllowedForMerge(context, fraction);
|
||||
}
|
||||
|
||||
public static long getMaxMemoryPerCompaction(TaskContextSupplier context, Properties properties) {
|
||||
if (properties.containsKey(MAX_MEMORY_FOR_COMPACTION_PROP)) {
|
||||
return Long.parseLong(properties.getProperty(MAX_MEMORY_FOR_COMPACTION_PROP));
|
||||
public static long getMaxMemoryPerCompaction(TaskContextSupplier context, HoodieConfig hoodieConfig) {
|
||||
if (hoodieConfig.contains(MAX_MEMORY_FOR_COMPACTION_PROP)) {
|
||||
return hoodieConfig.getLong(MAX_MEMORY_FOR_COMPACTION_PROP);
|
||||
}
|
||||
String fraction = properties.getProperty(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP, DEFAULT_MAX_MEMORY_FRACTION_FOR_COMPACTION);
|
||||
String fraction = hoodieConfig.getStringOrDefault(MAX_MEMORY_FRACTION_FOR_COMPACTION_PROP);
|
||||
return getMaxMemoryAllowedForMerge(context, fraction);
|
||||
}
|
||||
}
|
||||
|
||||
@@ -35,10 +35,10 @@ public abstract class BaseKeyGenerator extends KeyGenerator {
|
||||
|
||||
protected BaseKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
|
||||
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY,
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL));
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
this.hiveStylePartitioning = config.getBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.HIVE_STYLE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -32,9 +32,9 @@ public class ComplexAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
public ComplexAvroKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
|
||||
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY)
|
||||
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key())
|
||||
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
|
||||
@@ -55,8 +55,8 @@ public class CustomAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
public CustomAvroKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
this.partitionPathFields = Arrays.stream(props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()).split(",")).map(String::trim).collect(Collectors.toList());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -35,7 +35,7 @@ public class GlobalAvroDeleteKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
public GlobalAvroDeleteKeyGenerator(TypedProperties config) {
|
||||
super(config);
|
||||
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY).split(","));
|
||||
this.recordKeyFields = Arrays.asList(config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()).split(","));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -164,7 +164,7 @@ public class KeyGenUtils {
|
||||
*/
|
||||
public static KeyGenerator createKeyGeneratorByClassName(TypedProperties props) throws IOException {
|
||||
KeyGenerator keyGenerator = null;
|
||||
String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP, null);
|
||||
String keyGeneratorClass = props.getString(HoodieWriteConfig.KEYGENERATOR_CLASS_PROP.key(), null);
|
||||
if (!StringUtils.isNullOrEmpty(keyGeneratorClass)) {
|
||||
try {
|
||||
keyGenerator = (KeyGenerator) ReflectionUtils.loadClass(keyGeneratorClass, props);
|
||||
|
||||
@@ -36,7 +36,7 @@ public class NonpartitionedAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
public NonpartitionedAvroKeyGenerator(TypedProperties props) {
|
||||
super(props);
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY)
|
||||
this.recordKeyFields = Arrays.stream(props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key())
|
||||
.split(",")).map(String::trim).filter(s -> !s.isEmpty()).collect(Collectors.toList());
|
||||
this.partitionPathFields = EMPTY_PARTITION_FIELD_LIST;
|
||||
}
|
||||
|
||||
@@ -29,8 +29,8 @@ import java.util.Collections;
|
||||
public class SimpleAvroKeyGenerator extends BaseKeyGenerator {
|
||||
|
||||
public SimpleAvroKeyGenerator(TypedProperties props) {
|
||||
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
|
||||
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
|
||||
this(props, props.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
|
||||
props.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
|
||||
}
|
||||
|
||||
SimpleAvroKeyGenerator(TypedProperties props, String partitionPathField) {
|
||||
|
||||
@@ -88,8 +88,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
|
||||
}
|
||||
|
||||
public TimestampBasedAvroKeyGenerator(TypedProperties config) throws IOException {
|
||||
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY),
|
||||
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY));
|
||||
this(config, config.getString(KeyGeneratorOptions.RECORDKEY_FIELD_OPT_KEY.key()),
|
||||
config.getString(KeyGeneratorOptions.PARTITIONPATH_FIELD_OPT_KEY.key()));
|
||||
}
|
||||
|
||||
TimestampBasedAvroKeyGenerator(TypedProperties config, String partitionPathField) throws IOException {
|
||||
@@ -119,8 +119,8 @@ public class TimestampBasedAvroKeyGenerator extends SimpleAvroKeyGenerator {
|
||||
default:
|
||||
timeUnit = null;
|
||||
}
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY,
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL));
|
||||
this.encodePartitionPath = config.getBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.key(),
|
||||
Boolean.parseBoolean(KeyGeneratorOptions.URL_ENCODE_PARTITIONING_OPT_KEY.defaultValue()));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
||||
@@ -18,24 +18,33 @@
|
||||
|
||||
package org.apache.hudi.keygen.constant;
|
||||
|
||||
import org.apache.hudi.common.config.ConfigProperty;
|
||||
|
||||
public class KeyGeneratorOptions {
|
||||
|
||||
/**
|
||||
* Flag to indicate whether to use Hive style partitioning.
|
||||
* If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.
|
||||
* By default false (the names of partition folders are only partition values)
|
||||
*/
|
||||
public static final String URL_ENCODE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.partitionpath.urlencode";
|
||||
public static final String DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = "false";
|
||||
public static final String HIVE_STYLE_PARTITIONING_OPT_KEY = "hoodie.datasource.write.hive_style_partitioning";
|
||||
public static final String DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = "false";
|
||||
public static final ConfigProperty<String> URL_ENCODE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.urlencode")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("");
|
||||
|
||||
/**
|
||||
* Record key field. Value to be used as the `recordKey` component of `HoodieKey`. Actual value
|
||||
* will be obtained by invoking .toString() on the field value. Nested fields can be specified using
|
||||
* the dot notation eg: `a.b.c`
|
||||
*/
|
||||
public static final String RECORDKEY_FIELD_OPT_KEY = "hoodie.datasource.write.recordkey.field";
|
||||
public static final String PARTITIONPATH_FIELD_OPT_KEY = "hoodie.datasource.write.partitionpath.field";
|
||||
public static final ConfigProperty<String> HIVE_STYLE_PARTITIONING_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.hive_style_partitioning")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Flag to indicate whether to use Hive style partitioning.\n"
|
||||
+ "If set true, the names of partition folders follow <partition_column_name>=<partition_value> format.\n"
|
||||
+ "By default false (the names of partition folders are only partition values)");
|
||||
|
||||
public static final ConfigProperty<String> RECORDKEY_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.recordkey.field")
|
||||
.defaultValue("uuid")
|
||||
.withDocumentation("Record key field. Value to be used as the `recordKey` component of `HoodieKey`.\n"
|
||||
+ "Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using\n"
|
||||
+ "the dot notation eg: `a.b.c`");
|
||||
|
||||
public static final ConfigProperty<String> PARTITIONPATH_FIELD_OPT_KEY = ConfigProperty
|
||||
.key("hoodie.datasource.write.partitionpath.field")
|
||||
.defaultValue("partitionpath")
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
}
|
||||
|
||||
|
||||
@@ -50,7 +50,7 @@ public class HoodieAvroKeyGeneratorFactory {
|
||||
private static KeyGenerator createAvroKeyGeneratorByType(TypedProperties props) throws IOException {
|
||||
// Use KeyGeneratorType.SIMPLE as default keyGeneratorType
|
||||
String keyGeneratorType =
|
||||
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP, KeyGeneratorType.SIMPLE.name());
|
||||
props.getString(HoodieWriteConfig.KEYGENERATOR_TYPE_PROP.key(), KeyGeneratorType.SIMPLE.name());
|
||||
|
||||
KeyGeneratorType keyGeneratorTypeEnum;
|
||||
try {
|
||||
|
||||
@@ -67,7 +67,7 @@ import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
import static org.apache.hudi.common.table.HoodieTableConfig.DEFAULT_ARCHIVELOG_FOLDER;
|
||||
import static org.apache.hudi.common.table.HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX;
|
||||
import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP;
|
||||
|
||||
@@ -295,7 +295,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
|
||||
HoodieTableMetaClient.withPropertyBuilder()
|
||||
.setTableType(HoodieTableType.MERGE_ON_READ)
|
||||
.setTableName(tableName)
|
||||
.setArchiveLogFolder(DEFAULT_ARCHIVELOG_FOLDER)
|
||||
.setArchiveLogFolder(HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())
|
||||
.setPayloadClassName(HoodieMetadataPayload.class.getName())
|
||||
.setBaseFileFormat(HoodieFileFormat.HFILE.toString())
|
||||
.initTable(hadoopConf.get(), metadataWriteConfig.getBasePath());
|
||||
|
||||
@@ -111,7 +111,7 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
|
||||
HoodieMetadataConfig metadataConfig = HoodieMetadataConfig.newBuilder().fromProperties(config.getMetadataConfig().getProps())
|
||||
.build();
|
||||
this.metadata = HoodieTableMetadata.create(context, metadataConfig, config.getBasePath(),
|
||||
FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR);
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
|
||||
|
||||
this.viewManager = FileSystemViewManager.createViewManager(context, config.getMetadataConfig(), config.getViewStorageConfig(), () -> metadata);
|
||||
this.metaClient = metaClient;
|
||||
|
||||
@@ -204,7 +204,7 @@ public class HoodieTimelineArchiveLog<T extends HoodieAvroPayload, I, K, O> {
|
||||
// instant on the metadata table. This is required for metadata table sync.
|
||||
if (config.useFileListingMetadata()) {
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), config.getMetadataConfig(),
|
||||
config.getBasePath(), FileSystemViewStorageConfig.DEFAULT_VIEW_SPILLABLE_DIR)) {
|
||||
config.getBasePath(), FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
Option<String> lastSyncedInstantTime = tableMetadata.getSyncedInstantTime();
|
||||
|
||||
if (lastSyncedInstantTime.isPresent()) {
|
||||
|
||||
@@ -122,7 +122,7 @@ public abstract class AbstractUpgradeDowngrade {
|
||||
|
||||
// Write out the current version in hoodie.properties.updated file
|
||||
metaClient.getTableConfig().setTableVersion(toVersion);
|
||||
createUpdatedFile(metaClient.getTableConfig().getProperties());
|
||||
createUpdatedFile(metaClient.getTableConfig().getProps());
|
||||
|
||||
// because for different fs the fs.rename have different action,such as:
|
||||
// a) for hdfs : if propsFilePath already exist,fs.rename will not replace propsFilePath, but just return false
|
||||
|
||||
Reference in New Issue
Block a user