Restore 0.8.0 config keys with deprecated annotation (#3506)
Co-authored-by: Sagar Sumit <sagarsumit09@gmail.com> Co-authored-by: Vinoth Chandar <vinoth@apache.org>
This commit is contained in:
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.config;
|
||||
|
||||
import javax.annotation.concurrent.Immutable;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
@@ -38,21 +39,21 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
public static final String METADATA_PREFIX = "hoodie.metadata";
|
||||
|
||||
// Enable the internal Metadata Table which saves file listings
|
||||
public static final ConfigProperty<Boolean> METADATA_ENABLE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable the internal metadata table which serves table metadata like level file listings");
|
||||
|
||||
// Enable syncing the Metadata Table
|
||||
public static final ConfigProperty<Boolean> METADATA_SYNC_ENABLE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> SYNC_ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".sync.enable")
|
||||
.defaultValue(true)
|
||||
.sinceVersion("0.9.0")
|
||||
.withDocumentation("Enable syncing of metadata table from actions on the dataset");
|
||||
|
||||
// Validate contents of Metadata Table on each access against the actual filesystem
|
||||
public static final ConfigProperty<Boolean> METADATA_VALIDATE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> VALIDATE_ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".validate")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
@@ -61,68 +62,68 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
|
||||
|
||||
// Enable metrics for internal Metadata Table
|
||||
public static final ConfigProperty<Boolean> METADATA_METRICS_ENABLE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> METRICS_ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".metrics.enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable publishing of metrics around metadata table.");
|
||||
|
||||
// Parallelism for inserts
|
||||
public static final ConfigProperty<Integer> METADATA_INSERT_PARALLELISM_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> INSERT_PARALLELISM_VALUE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".insert.parallelism")
|
||||
.defaultValue(1)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Parallelism to use when inserting to the metadata table");
|
||||
|
||||
// Async clean
|
||||
public static final ConfigProperty<Boolean> METADATA_ASYNC_CLEAN_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> ASYNC_CLEAN_ENABLE = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".clean.async")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable asynchronous cleaning for metadata table");
|
||||
|
||||
// Maximum delta commits before compaction occurs
|
||||
public static final ConfigProperty<Integer> METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> COMPACT_NUM_DELTA_COMMITS = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".compact.max.delta.commits")
|
||||
.defaultValue(24)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls how often the metadata table is compacted.");
|
||||
|
||||
// Archival settings
|
||||
public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> MIN_COMMITS_TO_KEEP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".keep.min.commits")
|
||||
.defaultValue(20)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline.");
|
||||
|
||||
public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".keep.max.commits")
|
||||
.defaultValue(30)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline.");
|
||||
|
||||
// Cleaner commits retained
|
||||
public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".cleaner.commits.retained")
|
||||
.defaultValue(3)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls retention/history for metadata table.");
|
||||
|
||||
// Regex to filter out matching directories during bootstrap
|
||||
public static final ConfigProperty<String> DIRECTORY_FILTER_REGEX = ConfigProperty
|
||||
public static final ConfigProperty<String> DIR_FILTER_REGEX = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".dir.filter.regex")
|
||||
.defaultValue("")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Directories matching this regex, will be filtered out when initializing metadata table from lake storage for the first time.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_ASSUME_DATE_PARTITIONING_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> ASSUME_DATE_PARTITIONING = ConfigProperty
|
||||
.key("hoodie.assume.date.partitioning")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.3.0")
|
||||
.withDocumentation("Should HoodieWriteClient assume the data is partitioned by dates, i.e three levels from base path. "
|
||||
+ "This is a stop-gap to support tables created by versions < 0.3.1. Will be removed eventually");
|
||||
|
||||
public static final ConfigProperty<Integer> FILE_LISTING_PARALLELISM_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> FILE_LISTING_PARALLELISM_VALUE = ConfigProperty
|
||||
.key("hoodie.file.listing.parallelism")
|
||||
.defaultValue(1500)
|
||||
.sinceVersion("0.7.0")
|
||||
@@ -137,31 +138,31 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public int getFileListingParallelism() {
|
||||
return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_PROP), 1);
|
||||
return Math.max(getInt(HoodieMetadataConfig.FILE_LISTING_PARALLELISM_VALUE), 1);
|
||||
}
|
||||
|
||||
public Boolean shouldAssumeDatePartitioning() {
|
||||
return getBoolean(HoodieMetadataConfig.HOODIE_ASSUME_DATE_PARTITIONING_PROP);
|
||||
return getBoolean(HoodieMetadataConfig.ASSUME_DATE_PARTITIONING);
|
||||
}
|
||||
|
||||
public boolean enabled() {
|
||||
return getBoolean(METADATA_ENABLE_PROP);
|
||||
return getBoolean(ENABLE);
|
||||
}
|
||||
|
||||
public boolean enableSync() {
|
||||
return enabled() && getBoolean(HoodieMetadataConfig.METADATA_SYNC_ENABLE_PROP);
|
||||
return enabled() && getBoolean(HoodieMetadataConfig.SYNC_ENABLE);
|
||||
}
|
||||
|
||||
public boolean validateFileListingMetadata() {
|
||||
return getBoolean(METADATA_VALIDATE_PROP);
|
||||
return getBoolean(VALIDATE_ENABLE);
|
||||
}
|
||||
|
||||
public boolean enableMetrics() {
|
||||
return getBoolean(METADATA_METRICS_ENABLE_PROP);
|
||||
return getBoolean(METRICS_ENABLE);
|
||||
}
|
||||
|
||||
public String getDirectoryFilterRegex() {
|
||||
return getString(DIRECTORY_FILTER_REGEX);
|
||||
return getString(DIR_FILTER_REGEX);
|
||||
}
|
||||
|
||||
public static class Builder {
|
||||
@@ -181,63 +182,63 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Builder enable(boolean enable) {
|
||||
metadataConfig.setValue(METADATA_ENABLE_PROP, String.valueOf(enable));
|
||||
metadataConfig.setValue(ENABLE, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder enableSync(boolean enable) {
|
||||
metadataConfig.setValue(METADATA_SYNC_ENABLE_PROP, String.valueOf(enable));
|
||||
metadataConfig.setValue(SYNC_ENABLE, String.valueOf(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder enableMetrics(boolean enableMetrics) {
|
||||
metadataConfig.setValue(METADATA_METRICS_ENABLE_PROP, String.valueOf(enableMetrics));
|
||||
metadataConfig.setValue(METRICS_ENABLE, String.valueOf(enableMetrics));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder validate(boolean validate) {
|
||||
metadataConfig.setValue(METADATA_VALIDATE_PROP, String.valueOf(validate));
|
||||
metadataConfig.setValue(VALIDATE_ENABLE, String.valueOf(validate));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInsertParallelism(int parallelism) {
|
||||
metadataConfig.setValue(METADATA_INSERT_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
metadataConfig.setValue(INSERT_PARALLELISM_VALUE, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAsyncClean(boolean asyncClean) {
|
||||
metadataConfig.setValue(METADATA_ASYNC_CLEAN_PROP, String.valueOf(asyncClean));
|
||||
metadataConfig.setValue(ASYNC_CLEAN_ENABLE, String.valueOf(asyncClean));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxNumDeltaCommitsBeforeCompaction(int maxNumDeltaCommitsBeforeCompaction) {
|
||||
metadataConfig.setValue(METADATA_COMPACT_NUM_DELTA_COMMITS_PROP, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
metadataConfig.setValue(COMPACT_NUM_DELTA_COMMITS, String.valueOf(maxNumDeltaCommitsBeforeCompaction));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder archiveCommitsWith(int minToKeep, int maxToKeep) {
|
||||
metadataConfig.setValue(MIN_COMMITS_TO_KEEP_PROP, String.valueOf(minToKeep));
|
||||
metadataConfig.setValue(MAX_COMMITS_TO_KEEP_PROP, String.valueOf(maxToKeep));
|
||||
metadataConfig.setValue(MIN_COMMITS_TO_KEEP, String.valueOf(minToKeep));
|
||||
metadataConfig.setValue(MAX_COMMITS_TO_KEEP, String.valueOf(maxToKeep));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder retainCommits(int commitsRetained) {
|
||||
metadataConfig.setValue(CLEANER_COMMITS_RETAINED_PROP, String.valueOf(commitsRetained));
|
||||
metadataConfig.setValue(CLEANER_COMMITS_RETAINED, String.valueOf(commitsRetained));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withFileListingParallelism(int parallelism) {
|
||||
metadataConfig.setValue(FILE_LISTING_PARALLELISM_PROP, String.valueOf(parallelism));
|
||||
metadataConfig.setValue(FILE_LISTING_PARALLELISM_VALUE, String.valueOf(parallelism));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withAssumeDatePartitioning(boolean assumeDatePartitioning) {
|
||||
metadataConfig.setValue(HOODIE_ASSUME_DATE_PARTITIONING_PROP, String.valueOf(assumeDatePartitioning));
|
||||
metadataConfig.setValue(ASSUME_DATE_PARTITIONING, String.valueOf(assumeDatePartitioning));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withDirectoryFilterRegex(String regex) {
|
||||
metadataConfig.setValue(DIRECTORY_FILTER_REGEX, regex);
|
||||
metadataConfig.setValue(DIR_FILTER_REGEX, regex);
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -246,4 +247,140 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
return metadataConfig;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_ENABLE_PROP = ENABLE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final boolean DEFAULT_METADATA_ENABLE = ENABLE.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #VALIDATE_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_VALIDATE_PROP = VALIDATE_ENABLE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #VALIDATE_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final boolean DEFAULT_METADATA_VALIDATE = VALIDATE_ENABLE.defaultValue();
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #METRICS_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_METRICS_ENABLE_PROP = METRICS_ENABLE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #METRICS_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final boolean DEFAULT_METADATA_METRICS_ENABLE = METRICS_ENABLE.defaultValue();
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_INSERT_PARALLELISM_PROP = INSERT_PARALLELISM_VALUE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #INSERT_PARALLELISM_VALUE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_METADATA_INSERT_PARALLELISM = INSERT_PARALLELISM_VALUE.defaultValue();
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #ASYNC_CLEAN_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_ASYNC_CLEAN_PROP = ASYNC_CLEAN_ENABLE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #ASYNC_CLEAN_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final boolean DEFAULT_METADATA_ASYNC_CLEAN = ASYNC_CLEAN_ENABLE.defaultValue();
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #COMPACT_NUM_DELTA_COMMITS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String METADATA_COMPACT_NUM_DELTA_COMMITS_PROP = COMPACT_NUM_DELTA_COMMITS.key();
|
||||
/**
|
||||
* @deprecated Use {@link #COMPACT_NUM_DELTA_COMMITS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_METADATA_COMPACT_NUM_DELTA_COMMITS = COMPACT_NUM_DELTA_COMMITS.defaultValue();
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String MIN_COMMITS_TO_KEEP_PROP = MIN_COMMITS_TO_KEEP.key();
|
||||
/**
|
||||
* @deprecated Use {@link #MIN_COMMITS_TO_KEEP} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_MIN_COMMITS_TO_KEEP = MIN_COMMITS_TO_KEEP.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String MAX_COMMITS_TO_KEEP_PROP = MAX_COMMITS_TO_KEEP.key();
|
||||
/**
|
||||
* @deprecated Use {@link #MAX_COMMITS_TO_KEEP} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_MAX_COMMITS_TO_KEEP = MAX_COMMITS_TO_KEEP.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String CLEANER_COMMITS_RETAINED_PROP = CLEANER_COMMITS_RETAINED.key();
|
||||
/**
|
||||
* @deprecated Use {@link #CLEANER_COMMITS_RETAINED} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_CLEANER_COMMITS_RETAINED = CLEANER_COMMITS_RETAINED.defaultValue();
|
||||
/**
|
||||
* @deprecated No longer takes any effect.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String ENABLE_FALLBACK_PROP = METADATA_PREFIX + ".fallback.enable";
|
||||
/**
|
||||
* @deprecated No longer takes any effect.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_ENABLE_FALLBACK = "true";
|
||||
/**
|
||||
* @deprecated Use {@link #DIR_FILTER_REGEX} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DIRECTORY_FILTER_REGEX = DIR_FILTER_REGEX.key();
|
||||
/**
|
||||
* @deprecated Use {@link #DIR_FILTER_REGEX} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_DIRECTORY_FILTER_REGEX = DIR_FILTER_REGEX.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #ASSUME_DATE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_ASSUME_DATE_PARTITIONING_PROP = ASSUME_DATE_PARTITIONING.key();
|
||||
/**
|
||||
* @deprecated Use {@link #ASSUME_DATE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_ASSUME_DATE_PARTITIONING = ASSUME_DATE_PARTITIONING.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #FILE_LISTING_PARALLELISM_VALUE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILE_LISTING_PARALLELISM_PROP = FILE_LISTING_PARALLELISM_VALUE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #FILE_LISTING_PARALLELISM_VALUE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final int DEFAULT_FILE_LISTING_PARALLELISM = FILE_LISTING_PARALLELISM_VALUE.defaultValue();
|
||||
}
|
||||
|
||||
@@ -74,6 +74,60 @@ public class LockConfiguration implements Serializable {
|
||||
|
||||
public static final String ZK_LOCK_KEY_PROP_KEY = ZOOKEEPER_BASED_LOCK_PROPERTY_PREFIX + "lock_key";
|
||||
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_RETRY_MAX_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
@Deprecated
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_MAX_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(5000L);
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP = LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY;
|
||||
@Deprecated
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS = String.valueOf(10000L);
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_NUM_RETRIES_PROP = LOCK_ACQUIRE_NUM_RETRIES_PROP_KEY;
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP = LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY;
|
||||
@Deprecated
|
||||
public static final String DEFAULT_LOCK_ACQUIRE_CLIENT_NUM_RETRIES = String.valueOf(0);
|
||||
/** @deprecated Use {@link #LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP = LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY;
|
||||
@Deprecated
|
||||
public static final int DEFAULT_ACQUIRE_LOCK_WAIT_TIMEOUT_MS = 60 * 1000;
|
||||
/** @deprecated Use {@link #HIVE_DATABASE_NAME_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String HIVE_DATABASE_NAME_PROP = HIVE_DATABASE_NAME_PROP_KEY;
|
||||
/** @deprecated Use {@link #HIVE_TABLE_NAME_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String HIVE_TABLE_NAME_PROP = HIVE_TABLE_NAME_PROP_KEY;
|
||||
/** @deprecated Use {@link #HIVE_METASTORE_URI_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String HIVE_METASTORE_URI_PROP = HIVE_METASTORE_URI_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_BASE_PATH_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_BASE_PATH_PROP = ZK_BASE_PATH_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_SESSION_TIMEOUT_MS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_SESSION_TIMEOUT_MS_PROP = ZK_SESSION_TIMEOUT_MS_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_CONNECTION_TIMEOUT_MS_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_CONNECTION_TIMEOUT_MS_PROP = ZK_CONNECTION_TIMEOUT_MS_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_CONNECT_URL_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_CONNECT_URL_PROP = ZK_CONNECT_URL_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_PORT_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_PORT_PROP = ZK_PORT_PROP_KEY;
|
||||
/** @deprecated Use {@link #ZK_LOCK_KEY_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String ZK_LOCK_KEY_PROP = ZK_LOCK_KEY_PROP_KEY;
|
||||
|
||||
private final TypedProperties props;
|
||||
|
||||
public LockConfiguration(Properties props) {
|
||||
|
||||
@@ -37,7 +37,7 @@ import java.util.Properties;
|
||||
+ "(Tip: S3 is NOT eventually consistent anymore!)")
|
||||
public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
|
||||
public static final ConfigProperty<String> CONSISTENCY_CHECK_ENABLED_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> ENABLE = ConfigProperty
|
||||
.key("hoodie.consistency.check.enabled")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.5.0")
|
||||
@@ -45,14 +45,14 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
.withDocumentation("Enabled to handle S3 eventual consistency issue. This property is no longer required "
|
||||
+ "since S3 is now strongly consistent. Will be removed in the future releases.");
|
||||
|
||||
public static final ConfigProperty<Long> INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Long> INITIAL_CHECK_INTERVAL_MS = ConfigProperty
|
||||
.key("hoodie.consistency.check.initial_interval_ms")
|
||||
.defaultValue(400L)
|
||||
.sinceVersion("0.5.0")
|
||||
.deprecatedAfter("0.7.0")
|
||||
.withDocumentation("Amount of time (in ms) to wait, before checking for consistency after an operation on storage.");
|
||||
|
||||
public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Long> MAX_CHECK_INTERVAL_MS = ConfigProperty
|
||||
.key("hoodie.consistency.check.max_interval_ms")
|
||||
.defaultValue(20000L)
|
||||
.sinceVersion("0.5.0")
|
||||
@@ -60,7 +60,7 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
.withDocumentation("Maximum amount of time (in ms), to wait for consistency checking.");
|
||||
|
||||
// maximum number of checks, for consistency of written data. Will wait upto 140 Secs
|
||||
public static final ConfigProperty<Integer> MAX_CONSISTENCY_CHECKS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Integer> MAX_CHECKS = ConfigProperty
|
||||
.key("hoodie.consistency.check.max_checks")
|
||||
.defaultValue(6)
|
||||
.sinceVersion("0.5.0")
|
||||
@@ -68,14 +68,14 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
.withDocumentation("Maximum number of consistency checks to perform, with exponential backoff.");
|
||||
|
||||
// sleep time for OptimisticConsistencyGuard
|
||||
public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Long> OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS = ConfigProperty
|
||||
.key("hoodie.optimistic.consistency.guard.sleep_time_ms")
|
||||
.defaultValue(500L)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("Amount of time (in ms), to wait after which we assume storage is consistent.");
|
||||
|
||||
// config to enable OptimisticConsistencyGuard in finalizeWrite instead of FailSafeConsistencyGuard
|
||||
public static final ConfigProperty<Boolean> ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP = ConfigProperty
|
||||
public static final ConfigProperty<Boolean> OPTIMISTIC_CONSISTENCY_GUARD_ENABLE = ConfigProperty
|
||||
.key("_hoodie.optimistic.consistency.guard.enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.6.0")
|
||||
@@ -90,27 +90,27 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public boolean isConsistencyCheckEnabled() {
|
||||
return getBoolean(CONSISTENCY_CHECK_ENABLED_PROP);
|
||||
return getBoolean(ENABLE);
|
||||
}
|
||||
|
||||
public int getMaxConsistencyChecks() {
|
||||
return getInt(MAX_CONSISTENCY_CHECKS_PROP);
|
||||
return getInt(MAX_CHECKS);
|
||||
}
|
||||
|
||||
public int getInitialConsistencyCheckIntervalMs() {
|
||||
return getInt(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
|
||||
return getInt(INITIAL_CHECK_INTERVAL_MS);
|
||||
}
|
||||
|
||||
public int getMaxConsistencyCheckIntervalMs() {
|
||||
return getInt(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP);
|
||||
return getInt(MAX_CHECK_INTERVAL_MS);
|
||||
}
|
||||
|
||||
public long getOptimisticConsistencyGuardSleepTimeMs() {
|
||||
return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP);
|
||||
return getLong(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS);
|
||||
}
|
||||
|
||||
public boolean shouldEnableOptimisticConsistencyGuard() {
|
||||
return getBoolean(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP);
|
||||
return getBoolean(OPTIMISTIC_CONSISTENCY_GUARD_ENABLE);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -133,32 +133,32 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Builder withConsistencyCheckEnabled(boolean enabled) {
|
||||
consistencyGuardConfig.setValue(CONSISTENCY_CHECK_ENABLED_PROP, String.valueOf(enabled));
|
||||
consistencyGuardConfig.setValue(ENABLE, String.valueOf(enabled));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withInitialConsistencyCheckIntervalMs(int initialIntevalMs) {
|
||||
consistencyGuardConfig.setValue(INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(initialIntevalMs));
|
||||
consistencyGuardConfig.setValue(INITIAL_CHECK_INTERVAL_MS, String.valueOf(initialIntevalMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxConsistencyCheckIntervalMs(int maxIntervalMs) {
|
||||
consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP, String.valueOf(maxIntervalMs));
|
||||
consistencyGuardConfig.setValue(MAX_CHECK_INTERVAL_MS, String.valueOf(maxIntervalMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxConsistencyChecks(int maxConsistencyChecks) {
|
||||
consistencyGuardConfig.setValue(MAX_CONSISTENCY_CHECKS_PROP, String.valueOf(maxConsistencyChecks));
|
||||
consistencyGuardConfig.setValue(MAX_CHECKS, String.valueOf(maxConsistencyChecks));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withOptimisticConsistencyGuardSleepTimeMs(long sleepTimeMs) {
|
||||
consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP, String.valueOf(sleepTimeMs));
|
||||
consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS, String.valueOf(sleepTimeMs));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEnableOptimisticConsistencyGuard(boolean enableOptimisticConsistencyGuard) {
|
||||
consistencyGuardConfig.setValue(ENABLE_OPTIMISTIC_CONSISTENCY_GUARD_PROP, String.valueOf(enableOptimisticConsistencyGuard));
|
||||
consistencyGuardConfig.setValue(OPTIMISTIC_CONSISTENCY_GUARD_ENABLE, String.valueOf(enableOptimisticConsistencyGuard));
|
||||
return this;
|
||||
}
|
||||
|
||||
@@ -167,4 +167,65 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
return consistencyGuardConfig;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated use {@link #ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String CONSISTENCY_CHECK_ENABLED_PROP = ENABLE.key();
|
||||
/**
|
||||
* @deprecated use {@link #ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String DEFAULT_CONSISTENCY_CHECK_ENABLED = ENABLE.defaultValue();
|
||||
/**
|
||||
* @deprecated use {@link #INITIAL_CHECK_INTERVAL_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String INITIAL_CONSISTENCY_CHECK_INTERVAL_MS_PROP = INITIAL_CHECK_INTERVAL_MS.key();
|
||||
/**
|
||||
* @deprecated use {@link #INITIAL_CHECK_INTERVAL_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static long DEFAULT_INITIAL_CONSISTENCY_CHECK_INTERVAL_MS = INITIAL_CHECK_INTERVAL_MS.defaultValue();
|
||||
/**
|
||||
* @deprecated use {@link #MAX_CHECK_INTERVAL_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = MAX_CHECK_INTERVAL_MS.key();
|
||||
/**
|
||||
* @deprecated use {@link #MAX_CHECK_INTERVAL_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static long DEFAULT_MAX_CONSISTENCY_CHECK_INTERVAL_MS = MAX_CHECK_INTERVAL_MS.defaultValue();
|
||||
/**
|
||||
* @deprecated use {@link #MAX_CHECKS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String MAX_CONSISTENCY_CHECKS_PROP = MAX_CHECKS.key();
|
||||
/**
|
||||
* @deprecated use {@link #MAX_CHECKS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static int DEFAULT_MAX_CONSISTENCY_CHECKS = MAX_CHECKS.defaultValue();
|
||||
/**
|
||||
* @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS.key();
|
||||
/**
|
||||
* @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static long DEFAULT_OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS_PROP = OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS.defaultValue();
|
||||
/**
|
||||
* @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = OPTIMISTIC_CONSISTENCY_GUARD_ENABLE.key();
|
||||
/**
|
||||
* @deprecated use {@link #OPTIMISTIC_CONSISTENCY_GUARD_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static boolean DEFAULT_ENABLE_OPTIMISTIC_CONSISTENCY_GUARD = OPTIMISTIC_CONSISTENCY_GUARD_ENABLE.defaultValue();
|
||||
}
|
||||
|
||||
@@ -18,14 +18,6 @@
|
||||
|
||||
package org.apache.hudi.common.fs;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig;
|
||||
import org.apache.hudi.common.config.SerializableConfiguration;
|
||||
import org.apache.hudi.common.engine.HoodieEngineContext;
|
||||
@@ -42,6 +34,15 @@ import org.apache.hudi.exception.HoodieException;
|
||||
import org.apache.hudi.exception.HoodieIOException;
|
||||
import org.apache.hudi.exception.InvalidHoodiePathException;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.LocatedFileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.fs.RemoteIterator;
|
||||
import org.apache.hadoop.hdfs.DistributedFileSystem;
|
||||
import org.apache.log4j.LogManager;
|
||||
import org.apache.log4j.Logger;
|
||||
|
||||
@@ -141,7 +142,7 @@ public class FSUtils {
|
||||
// TODO: this should be removed
|
||||
public static String makeDataFileName(String instantTime, String writeToken, String fileId) {
|
||||
return String.format("%s_%s_%s%s", fileId, writeToken, instantTime,
|
||||
HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
|
||||
HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
|
||||
}
|
||||
|
||||
public static String makeDataFileName(String instantTime, String writeToken, String fileId, String fileExtension) {
|
||||
@@ -153,7 +154,7 @@ public class FSUtils {
|
||||
}
|
||||
|
||||
public static String maskWithoutFileId(String instantTime, int taskPartitionId) {
|
||||
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP
|
||||
return String.format("*_%s_%s%s", taskPartitionId, instantTime, HoodieTableConfig.BASE_FILE_FORMAT
|
||||
.defaultValue().getFileExtension());
|
||||
}
|
||||
|
||||
@@ -271,7 +272,7 @@ public class FSUtils {
|
||||
.withAssumeDatePartitioning(assumeDatePartitioning)
|
||||
.build();
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
|
||||
return tableMetadata.getAllPartitionPaths();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error fetching partition paths from metadata table", e);
|
||||
@@ -281,7 +282,7 @@ public class FSUtils {
|
||||
public static List<String> getAllPartitionPaths(HoodieEngineContext engineContext, HoodieMetadataConfig metadataConfig,
|
||||
String basePathStr) {
|
||||
try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, basePathStr,
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue())) {
|
||||
FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue())) {
|
||||
return tableMetadata.getAllPartitionPaths();
|
||||
} catch (Exception e) {
|
||||
throw new HoodieException("Error fetching partition paths from metadata table", e);
|
||||
|
||||
@@ -48,4 +48,15 @@ public class HoodiePayloadProps {
|
||||
* to tell if the current record is a update record or insert record for mor table.
|
||||
*/
|
||||
public static final String PAYLOAD_IS_UPDATE_RECORD_FOR_MOR = "hoodie.is.update.record.for.mor";
|
||||
|
||||
/** @deprecated Use {@link #PAYLOAD_ORDERING_FIELD_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String PAYLOAD_ORDERING_FIELD_PROP = PAYLOAD_ORDERING_FIELD_PROP_KEY;
|
||||
@Deprecated
|
||||
public static String DEFAULT_PAYLOAD_ORDERING_FIELD_VAL = "ts";
|
||||
/** @deprecated Use {@link #PAYLOAD_EVENT_TIME_FIELD_PROP_KEY} */
|
||||
@Deprecated
|
||||
public static final String PAYLOAD_EVENT_TIME_FIELD_PROP = PAYLOAD_EVENT_TIME_FIELD_PROP_KEY;
|
||||
@Deprecated
|
||||
public static String DEFAULT_PAYLOAD_EVENT_TIME_FIELD_VAL = "ts";
|
||||
}
|
||||
|
||||
@@ -69,95 +69,95 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
|
||||
public static final String HOODIE_PROPERTIES_FILE = "hoodie.properties";
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_NAME_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> NAME = ConfigProperty
|
||||
.key("hoodie.table.name")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Table name that will be used for registering with Hive. Needs to be same across runs.");
|
||||
|
||||
public static final ConfigProperty<HoodieTableType> HOODIE_TABLE_TYPE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<HoodieTableType> TYPE = ConfigProperty
|
||||
.key("hoodie.table.type")
|
||||
.defaultValue(HoodieTableType.COPY_ON_WRITE)
|
||||
.withDocumentation("The table type for the underlying data, for this write. This can’t change between writes.");
|
||||
|
||||
public static final ConfigProperty<HoodieTableVersion> HOODIE_TABLE_VERSION_PROP = ConfigProperty
|
||||
public static final ConfigProperty<HoodieTableVersion> VERSION = ConfigProperty
|
||||
.key("hoodie.table.version")
|
||||
.defaultValue(HoodieTableVersion.ZERO)
|
||||
.withDocumentation("Version of table, used for running upgrade/downgrade steps between releases with potentially "
|
||||
+ "breaking/backwards compatible changes.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_PRECOMBINE_FIELD_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> PRECOMBINE_FIELD = ConfigProperty
|
||||
.key("hoodie.table.precombine.field")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Field used in preCombining before actual write. By default, when two records have the same key value, "
|
||||
+ "the largest value for the precombine field determined by Object.compareTo(..), is picked.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_PARTITION_FIELDS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> PARTITION_FIELDS = ConfigProperty
|
||||
.key("hoodie.table.partition.fields")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Fields used to partition the table. Concatenated values of these fields are used as "
|
||||
+ "the partition path, by invoking toString()");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_RECORDKEY_FIELDS = ConfigProperty
|
||||
public static final ConfigProperty<String> RECORDKEY_FIELDS = ConfigProperty
|
||||
.key("hoodie.table.recordkey.fields")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Columns used to uniquely identify the table. Concatenated values of these fields are used as "
|
||||
+ " the record key component of HoodieKey.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_CREATE_SCHEMA = ConfigProperty
|
||||
public static final ConfigProperty<String> CREATE_SCHEMA = ConfigProperty
|
||||
.key("hoodie.table.create.schema")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Schema used when creating the table, for the first time.");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty
|
||||
public static final ConfigProperty<HoodieFileFormat> BASE_FILE_FORMAT = ConfigProperty
|
||||
.key("hoodie.table.base.file.format")
|
||||
.defaultValue(HoodieFileFormat.PARQUET)
|
||||
.withAlternatives("hoodie.table.ro.file.format")
|
||||
.withDocumentation("Base file format to store all the base file data.");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_LOG_FILE_FORMAT_PROP = ConfigProperty
|
||||
public static final ConfigProperty<HoodieFileFormat> LOG_FILE_FORMAT = ConfigProperty
|
||||
.key("hoodie.table.log.file.format")
|
||||
.defaultValue(HoodieFileFormat.HOODIE_LOG)
|
||||
.withAlternatives("hoodie.table.rt.file.format")
|
||||
.withDocumentation("Log format used for the delta logs.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TIMELINE_LAYOUT_VERSION_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> TIMELINE_LAYOUT_VERSION = ConfigProperty
|
||||
.key("hoodie.timeline.layout.version")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Version of timeline used, by the table.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_PAYLOAD_CLASS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> PAYLOAD_CLASS_NAME = ConfigProperty
|
||||
.key("hoodie.compaction.payload.class")
|
||||
.defaultValue(OverwriteWithLatestAvroPayload.class.getName())
|
||||
.withDocumentation("Payload class to use for performing compactions, i.e merge delta logs with current base file and then "
|
||||
+ " produce a new base file.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_ARCHIVELOG_FOLDER_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> ARCHIVELOG_FOLDER = ConfigProperty
|
||||
.key("hoodie.archivelog.folder")
|
||||
.defaultValue("archived")
|
||||
.withDocumentation("path under the meta folder, to store archived timeline instants at.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> BOOTSTRAP_INDEX_ENABLE = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.enable")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Whether or not, this is a bootstrapped table, with bootstrap base data and an mapping index defined.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_CLASS_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> BOOTSTRAP_INDEX_CLASS_NAME = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.class")
|
||||
.defaultValue(HFileBootstrapIndex.class.getName())
|
||||
.withDocumentation("Implementation to use, for mapping base files to bootstrap base file, that contain actual data.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_BASE_PATH_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> BOOTSTRAP_BASE_PATH = ConfigProperty
|
||||
.key("hoodie.bootstrap.base.path")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Base path of the dataset that needs to be bootstrapped as a Hudi table");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_POPULATE_META_FIELDS = ConfigProperty
|
||||
public static final ConfigProperty<String> POPULATE_META_FIELDS = ConfigProperty
|
||||
.key("hoodie.populate.meta.fields")
|
||||
.defaultValue("true")
|
||||
.withDocumentation("When enabled, populates all meta fields. When disabled, no meta fields are populated "
|
||||
+ "and incremental queries will not be functional. This is only meant to be used for append only/immutable data for batch processing");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TABLE_KEY_GENERATOR_CLASS = ConfigProperty
|
||||
public static final ConfigProperty<String> KEY_GENERATOR_CLASS_NAME = ConfigProperty
|
||||
.key("hoodie.table.keygenerator.class")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Key Generator class property for the hoodie table");
|
||||
@@ -172,9 +172,9 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
try (FSDataInputStream inputStream = fs.open(propertyPath)) {
|
||||
props.load(inputStream);
|
||||
}
|
||||
if (contains(HOODIE_PAYLOAD_CLASS_PROP) && payloadClassName != null
|
||||
&& !getString(HOODIE_PAYLOAD_CLASS_PROP).equals(payloadClassName)) {
|
||||
setValue(HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null
|
||||
&& !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) {
|
||||
setValue(PAYLOAD_CLASS_NAME, payloadClassName);
|
||||
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||
props.store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
@@ -182,7 +182,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
} catch (IOException e) {
|
||||
throw new HoodieIOException("Could not load Hoodie properties from " + propertyPath, e);
|
||||
}
|
||||
ValidationUtils.checkArgument(contains(HOODIE_TABLE_TYPE_PROP) && contains(HOODIE_TABLE_NAME_PROP),
|
||||
ValidationUtils.checkArgument(contains(TYPE) && contains(NAME),
|
||||
"hoodie.properties file seems invalid. Please check for left over `.updated` files if any, manually copy it to hoodie.properties and retry");
|
||||
}
|
||||
|
||||
@@ -205,21 +205,21 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
Path propertyPath = new Path(metadataFolder, HOODIE_PROPERTIES_FILE);
|
||||
try (FSDataOutputStream outputStream = fs.create(propertyPath)) {
|
||||
if (!hoodieConfig.contains(HOODIE_TABLE_NAME_PROP)) {
|
||||
throw new IllegalArgumentException(HOODIE_TABLE_NAME_PROP.key() + " property needs to be specified");
|
||||
if (!hoodieConfig.contains(NAME)) {
|
||||
throw new IllegalArgumentException(NAME.key() + " property needs to be specified");
|
||||
}
|
||||
hoodieConfig.setDefaultValue(HOODIE_TABLE_TYPE_PROP);
|
||||
if (hoodieConfig.getString(HOODIE_TABLE_TYPE_PROP).equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
hoodieConfig.setDefaultValue(HOODIE_PAYLOAD_CLASS_PROP);
|
||||
hoodieConfig.setDefaultValue(TYPE);
|
||||
if (hoodieConfig.getString(TYPE).equals(HoodieTableType.MERGE_ON_READ.name())) {
|
||||
hoodieConfig.setDefaultValue(PAYLOAD_CLASS_NAME);
|
||||
}
|
||||
hoodieConfig.setDefaultValue(HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
if (!hoodieConfig.contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
|
||||
hoodieConfig.setDefaultValue(ARCHIVELOG_FOLDER);
|
||||
if (!hoodieConfig.contains(TIMELINE_LAYOUT_VERSION)) {
|
||||
// Use latest Version as default unless forced by client
|
||||
hoodieConfig.setValue(HOODIE_TIMELINE_LAYOUT_VERSION_PROP, TimelineLayoutVersion.CURR_VERSION.toString());
|
||||
hoodieConfig.setValue(TIMELINE_LAYOUT_VERSION, TimelineLayoutVersion.CURR_VERSION.toString());
|
||||
}
|
||||
if (hoodieConfig.contains(HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
|
||||
if (hoodieConfig.contains(BOOTSTRAP_BASE_PATH)) {
|
||||
// Use the default bootstrap index class.
|
||||
hoodieConfig.setDefaultValue(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(properties));
|
||||
hoodieConfig.setDefaultValue(BOOTSTRAP_INDEX_CLASS_NAME, getDefaultBootstrapIndexClass(properties));
|
||||
}
|
||||
hoodieConfig.getProps().store(outputStream, "Properties saved on " + new Date(System.currentTimeMillis()));
|
||||
}
|
||||
@@ -229,12 +229,12 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* Read the table type from the table properties and if not found, return the default.
|
||||
*/
|
||||
public HoodieTableType getTableType() {
|
||||
return HoodieTableType.valueOf(getStringOrDefault(HOODIE_TABLE_TYPE_PROP));
|
||||
return HoodieTableType.valueOf(getStringOrDefault(TYPE));
|
||||
}
|
||||
|
||||
public Option<TimelineLayoutVersion> getTimelineLayoutVersion() {
|
||||
return contains(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)
|
||||
? Option.of(new TimelineLayoutVersion(getInt(HOODIE_TIMELINE_LAYOUT_VERSION_PROP)))
|
||||
return contains(TIMELINE_LAYOUT_VERSION)
|
||||
? Option.of(new TimelineLayoutVersion(getInt(TIMELINE_LAYOUT_VERSION)))
|
||||
: Option.empty();
|
||||
}
|
||||
|
||||
@@ -242,13 +242,13 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* @return the hoodie.table.version from hoodie.properties file.
|
||||
*/
|
||||
public HoodieTableVersion getTableVersion() {
|
||||
return contains(HOODIE_TABLE_VERSION_PROP)
|
||||
? HoodieTableVersion.versionFromCode(getInt(HOODIE_TABLE_VERSION_PROP))
|
||||
: HOODIE_TABLE_VERSION_PROP.defaultValue();
|
||||
return contains(VERSION)
|
||||
? HoodieTableVersion.versionFromCode(getInt(VERSION))
|
||||
: VERSION.defaultValue();
|
||||
}
|
||||
|
||||
public void setTableVersion(HoodieTableVersion tableVersion) {
|
||||
setValue(HOODIE_TABLE_VERSION_PROP, Integer.toString(tableVersion.versionCode()));
|
||||
setValue(VERSION, Integer.toString(tableVersion.versionCode()));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -257,26 +257,26 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
public String getPayloadClass() {
|
||||
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
|
||||
// change to org.apache.hudi
|
||||
return getStringOrDefault(HOODIE_PAYLOAD_CLASS_PROP).replace("com.uber.hoodie",
|
||||
return getStringOrDefault(PAYLOAD_CLASS_NAME).replace("com.uber.hoodie",
|
||||
"org.apache.hudi");
|
||||
}
|
||||
|
||||
public String getPreCombineField() {
|
||||
return getString(HOODIE_TABLE_PRECOMBINE_FIELD_PROP);
|
||||
return getString(PRECOMBINE_FIELD);
|
||||
}
|
||||
|
||||
public Option<String[]> getRecordKeyFields() {
|
||||
if (contains(HOODIE_TABLE_RECORDKEY_FIELDS)) {
|
||||
return Option.of(Arrays.stream(getString(HOODIE_TABLE_RECORDKEY_FIELDS).split(","))
|
||||
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
|
||||
if (contains(RECORDKEY_FIELDS)) {
|
||||
return Option.of(Arrays.stream(getString(RECORDKEY_FIELDS).split(","))
|
||||
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
public Option<String[]> getPartitionFields() {
|
||||
if (contains(HOODIE_TABLE_PARTITION_FIELDS_PROP)) {
|
||||
return Option.of(Arrays.stream(getString(HOODIE_TABLE_PARTITION_FIELDS_PROP).split(","))
|
||||
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[]{}));
|
||||
if (contains(PARTITION_FIELDS)) {
|
||||
return Option.of(Arrays.stream(getString(PARTITION_FIELDS).split(","))
|
||||
.filter(p -> p.length() > 0).collect(Collectors.toList()).toArray(new String[] {}));
|
||||
}
|
||||
return Option.empty();
|
||||
}
|
||||
@@ -285,7 +285,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* @returns the partition field prop.
|
||||
*/
|
||||
public String getPartitionFieldProp() {
|
||||
return getString(HOODIE_TABLE_PARTITION_FIELDS_PROP);
|
||||
return getString(PARTITION_FIELDS);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -294,24 +294,24 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
public String getBootstrapIndexClass() {
|
||||
// There could be tables written with payload class from com.uber.hoodie. Need to transparently
|
||||
// change to org.apache.hudi
|
||||
return getStringOrDefault(HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, getDefaultBootstrapIndexClass(props));
|
||||
return getStringOrDefault(BOOTSTRAP_INDEX_CLASS_NAME, getDefaultBootstrapIndexClass(props));
|
||||
}
|
||||
|
||||
public static String getDefaultBootstrapIndexClass(Properties props) {
|
||||
String defaultClass = HOODIE_BOOTSTRAP_INDEX_CLASS_PROP.defaultValue();
|
||||
if ("false".equalsIgnoreCase(props.getProperty(HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key()))) {
|
||||
String defaultClass = BOOTSTRAP_INDEX_CLASS_NAME.defaultValue();
|
||||
if ("false".equalsIgnoreCase(props.getProperty(BOOTSTRAP_INDEX_ENABLE.key()))) {
|
||||
defaultClass = NO_OP_BOOTSTRAP_INDEX_CLASS;
|
||||
}
|
||||
return defaultClass;
|
||||
}
|
||||
|
||||
public Option<String> getBootstrapBasePath() {
|
||||
return Option.ofNullable(getString(HOODIE_BOOTSTRAP_BASE_PATH_PROP));
|
||||
return Option.ofNullable(getString(BOOTSTRAP_BASE_PATH));
|
||||
}
|
||||
|
||||
public Option<Schema> getTableCreateSchema() {
|
||||
if (contains(HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
return Option.of(new Schema.Parser().parse(getString(HOODIE_TABLE_CREATE_SCHEMA)));
|
||||
if (contains(CREATE_SCHEMA)) {
|
||||
return Option.of(new Schema.Parser().parse(getString(CREATE_SCHEMA)));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
@@ -321,7 +321,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* Read the table name.
|
||||
*/
|
||||
public String getTableName() {
|
||||
return getString(HOODIE_TABLE_NAME_PROP);
|
||||
return getString(NAME);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -330,7 +330,7 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* @return HoodieFileFormat for the base file Storage format
|
||||
*/
|
||||
public HoodieFileFormat getBaseFileFormat() {
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_BASE_FILE_FORMAT_PROP));
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(BASE_FILE_FORMAT));
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -339,32 +339,133 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
* @return HoodieFileFormat for the log Storage format
|
||||
*/
|
||||
public HoodieFileFormat getLogFileFormat() {
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(HOODIE_LOG_FILE_FORMAT_PROP));
|
||||
return HoodieFileFormat.valueOf(getStringOrDefault(LOG_FILE_FORMAT));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the relative path of archive log folder under metafolder, for this table.
|
||||
*/
|
||||
public String getArchivelogFolder() {
|
||||
return getStringOrDefault(HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
return getStringOrDefault(ARCHIVELOG_FOLDER);
|
||||
}
|
||||
|
||||
/**
|
||||
* @returns true is meta fields need to be populated. else returns false.
|
||||
*/
|
||||
public boolean populateMetaFields() {
|
||||
return Boolean.parseBoolean(getStringOrDefault(HOODIE_POPULATE_META_FIELDS));
|
||||
return Boolean.parseBoolean(getStringOrDefault(POPULATE_META_FIELDS));
|
||||
}
|
||||
|
||||
/**
|
||||
* @returns the record key field prop.
|
||||
*/
|
||||
public String getRecordKeyFieldProp() {
|
||||
return getString(HOODIE_TABLE_RECORDKEY_FIELDS);
|
||||
return getString(RECORDKEY_FIELDS);
|
||||
}
|
||||
|
||||
public Map<String, String> propsMap() {
|
||||
return props.entrySet().stream()
|
||||
.collect(Collectors.toMap(e -> String.valueOf(e.getKey()), e -> String.valueOf(e.getValue())));
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_RO_FILE_FORMAT_PROP_NAME = "hoodie.table.ro.file.format";
|
||||
/**
|
||||
* @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_RT_FILE_FORMAT_PROP_NAME = "hoodie.table.rt.file.format";
|
||||
/**
|
||||
* @deprecated Use {@link #NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_TABLE_NAME_PROP_NAME = NAME.key();
|
||||
/**
|
||||
* @deprecated Use {@link #TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_TABLE_TYPE_PROP_NAME = TYPE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #VERSION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_TABLE_VERSION_PROP_NAME = VERSION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #PRECOMBINE_FIELD} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_TABLE_PRECOMBINE_FIELD = PRECOMBINE_FIELD.key();
|
||||
/**
|
||||
* @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_BASE_FILE_FORMAT_PROP_NAME = BASE_FILE_FORMAT.key();
|
||||
/**
|
||||
* @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_LOG_FILE_FORMAT_PROP_NAME = LOG_FILE_FORMAT.key();
|
||||
/**
|
||||
* @deprecated Use {@link #TIMELINE_LAYOUT_VERSION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_TIMELINE_LAYOUT_VERSION = TIMELINE_LAYOUT_VERSION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_PAYLOAD_CLASS_PROP_NAME = PAYLOAD_CLASS_NAME.key();
|
||||
/**
|
||||
* @deprecated Use {@link #ARCHIVELOG_FOLDER} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_ARCHIVELOG_FOLDER_PROP_NAME = ARCHIVELOG_FOLDER.key();
|
||||
/**
|
||||
* @deprecated Use {@link #BOOTSTRAP_INDEX_CLASS_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_BOOTSTRAP_INDEX_CLASS_PROP_NAME = BOOTSTRAP_INDEX_CLASS_NAME.key();
|
||||
/**
|
||||
* @deprecated Use {@link #BOOTSTRAP_BASE_PATH} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HOODIE_BOOTSTRAP_BASE_PATH = BOOTSTRAP_BASE_PATH.key();
|
||||
/**
|
||||
* @deprecated Use {@link #TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final HoodieTableType DEFAULT_TABLE_TYPE = TYPE.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #VERSION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final HoodieTableVersion DEFAULT_TABLE_VERSION = VERSION.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #BASE_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final HoodieFileFormat DEFAULT_BASE_FILE_FORMAT = BASE_FILE_FORMAT.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #LOG_FILE_FORMAT} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final HoodieFileFormat DEFAULT_LOG_FILE_FORMAT = LOG_FILE_FORMAT.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #PAYLOAD_CLASS_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_PAYLOAD_CLASS = PAYLOAD_CLASS_NAME.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #BOOTSTRAP_INDEX_CLASS_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_BOOTSTRAP_INDEX_CLASS = BOOTSTRAP_INDEX_CLASS_NAME.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #ARCHIVELOG_FOLDER} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_ARCHIVELOG_FOLDER = ARCHIVELOG_FOLDER.defaultValue();
|
||||
}
|
||||
|
||||
@@ -227,7 +227,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
*/
|
||||
public String getArchivePath() {
|
||||
String archiveFolder = tableConfig.getArchivelogFolder();
|
||||
if (archiveFolder.equals(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP.defaultValue())) {
|
||||
if (archiveFolder.equals(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue())) {
|
||||
return getMetaPath();
|
||||
} else {
|
||||
return getMetaPath() + "/" + archiveFolder;
|
||||
@@ -321,16 +321,16 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
public void validateTableProperties(Properties properties, WriteOperationType operationType) {
|
||||
// once meta fields are disabled, it cant be re-enabled for a given table.
|
||||
if (!getTableConfig().populateMetaFields()
|
||||
&& Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key(), HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.defaultValue()))) {
|
||||
throw new HoodieException(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
|
||||
&& Boolean.parseBoolean((String) properties.getOrDefault(HoodieTableConfig.POPULATE_META_FIELDS.key(), HoodieTableConfig.POPULATE_META_FIELDS.defaultValue()))) {
|
||||
throw new HoodieException(HoodieTableConfig.POPULATE_META_FIELDS.key() + " already disabled for the table. Can't be re-enabled back");
|
||||
}
|
||||
|
||||
// meta fields can be disabled only with SimpleKeyGenerator
|
||||
if (!getTableConfig().populateMetaFields()
|
||||
&& !properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key(), "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
&& !properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key(), "org.apache.hudi.keygen.SimpleKeyGenerator")
|
||||
.equals("org.apache.hudi.keygen.SimpleKeyGenerator")) {
|
||||
throw new HoodieException("Only simple key generator is supported when meta fields are disabled. KeyGenerator used : "
|
||||
+ properties.getProperty(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS.key()));
|
||||
+ properties.getProperty(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME.key()));
|
||||
}
|
||||
}
|
||||
|
||||
@@ -353,7 +353,7 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
}
|
||||
|
||||
// if anything other than default archive log folder is specified, create that too
|
||||
String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
String archiveLogPropVal = new HoodieConfig(props).getStringOrDefault(HoodieTableConfig.ARCHIVELOG_FOLDER);
|
||||
if (!StringUtils.isNullOrEmpty(archiveLogPropVal)) {
|
||||
Path archiveLogDir = new Path(metaPathDir, archiveLogPropVal);
|
||||
if (!fs.exists(archiveLogDir)) {
|
||||
@@ -725,52 +725,52 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
|
||||
public PropertyBuilder fromProperties(Properties properties) {
|
||||
HoodieConfig hoodieConfig = new HoodieConfig(properties);
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_NAME_PROP)) {
|
||||
setTableName(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_NAME_PROP));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.NAME)) {
|
||||
setTableName(hoodieConfig.getString(HoodieTableConfig.NAME));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP)) {
|
||||
setTableType(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.TYPE)) {
|
||||
setTableType(hoodieConfig.getString(HoodieTableConfig.TYPE));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.ARCHIVELOG_FOLDER)) {
|
||||
setArchiveLogFolder(
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP));
|
||||
hoodieConfig.getString(HoodieTableConfig.ARCHIVELOG_FOLDER));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.PAYLOAD_CLASS_NAME)) {
|
||||
setPayloadClassName(
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP));
|
||||
hoodieConfig.getString(HoodieTableConfig.PAYLOAD_CLASS_NAME));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP)) {
|
||||
setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.TIMELINE_LAYOUT_VERSION)) {
|
||||
setTimelineLayoutVersion(hoodieConfig.getInt(HoodieTableConfig.TIMELINE_LAYOUT_VERSION));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.BASE_FILE_FORMAT)) {
|
||||
setBaseFileFormat(
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP));
|
||||
hoodieConfig.getString(HoodieTableConfig.BASE_FILE_FORMAT));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME)) {
|
||||
setBootstrapIndexClass(
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP));
|
||||
hoodieConfig.getString(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP)) {
|
||||
setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.BOOTSTRAP_BASE_PATH)) {
|
||||
setBootstrapBasePath(hoodieConfig.getString(HoodieTableConfig.BOOTSTRAP_BASE_PATH));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP)) {
|
||||
setPreCombineField(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.PRECOMBINE_FIELD)) {
|
||||
setPreCombineField(hoodieConfig.getString(HoodieTableConfig.PRECOMBINE_FIELD));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP)) {
|
||||
if (hoodieConfig.contains(HoodieTableConfig.PARTITION_FIELDS)) {
|
||||
setPartitionFields(
|
||||
hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP));
|
||||
hoodieConfig.getString(HoodieTableConfig.PARTITION_FIELDS));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS)) {
|
||||
setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.RECORDKEY_FIELDS)) {
|
||||
setRecordKeyFields(hoodieConfig.getString(HoodieTableConfig.RECORDKEY_FIELDS));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA)) {
|
||||
setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.CREATE_SCHEMA)) {
|
||||
setTableCreateSchema(hoodieConfig.getString(HoodieTableConfig.CREATE_SCHEMA));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS)) {
|
||||
setPopulateMetaFields(hoodieConfig.getBoolean(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.POPULATE_META_FIELDS)) {
|
||||
setPopulateMetaFields(hoodieConfig.getBoolean(HoodieTableConfig.POPULATE_META_FIELDS));
|
||||
}
|
||||
if (hoodieConfig.contains(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS)) {
|
||||
setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS));
|
||||
if (hoodieConfig.contains(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME)) {
|
||||
setKeyGeneratorClassProp(hoodieConfig.getString(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME));
|
||||
}
|
||||
return this;
|
||||
}
|
||||
@@ -780,56 +780,56 @@ public class HoodieTableMetaClient implements Serializable {
|
||||
ValidationUtils.checkArgument(tableName != null, "tableName is null");
|
||||
|
||||
HoodieTableConfig tableConfig = new HoodieTableConfig();
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_NAME_PROP, tableName);
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_TYPE_PROP, tableType.name());
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_VERSION_PROP,
|
||||
tableConfig.setValue(HoodieTableConfig.NAME, tableName);
|
||||
tableConfig.setValue(HoodieTableConfig.TYPE, tableType.name());
|
||||
tableConfig.setValue(HoodieTableConfig.VERSION,
|
||||
String.valueOf(HoodieTableVersion.current().versionCode()));
|
||||
if (tableType == HoodieTableType.MERGE_ON_READ && payloadClassName != null) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_PAYLOAD_CLASS_PROP, payloadClassName);
|
||||
tableConfig.setValue(HoodieTableConfig.PAYLOAD_CLASS_NAME, payloadClassName);
|
||||
}
|
||||
|
||||
if (null != tableCreateSchema) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_CREATE_SCHEMA, tableCreateSchema);
|
||||
tableConfig.setValue(HoodieTableConfig.CREATE_SCHEMA, tableCreateSchema);
|
||||
}
|
||||
|
||||
if (!StringUtils.isNullOrEmpty(archiveLogFolder)) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP, archiveLogFolder);
|
||||
tableConfig.setValue(HoodieTableConfig.ARCHIVELOG_FOLDER, archiveLogFolder);
|
||||
} else {
|
||||
tableConfig.setDefaultValue(HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP);
|
||||
tableConfig.setDefaultValue(HoodieTableConfig.ARCHIVELOG_FOLDER);
|
||||
}
|
||||
|
||||
if (null != timelineLayoutVersion) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TIMELINE_LAYOUT_VERSION_PROP,
|
||||
tableConfig.setValue(HoodieTableConfig.TIMELINE_LAYOUT_VERSION,
|
||||
String.valueOf(timelineLayoutVersion));
|
||||
}
|
||||
|
||||
if (null != baseFileFormat) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP, baseFileFormat.toUpperCase());
|
||||
tableConfig.setValue(HoodieTableConfig.BASE_FILE_FORMAT, baseFileFormat.toUpperCase());
|
||||
}
|
||||
|
||||
if (null != bootstrapIndexClass) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_CLASS_PROP, bootstrapIndexClass);
|
||||
tableConfig.setValue(HoodieTableConfig.BOOTSTRAP_INDEX_CLASS_NAME, bootstrapIndexClass);
|
||||
}
|
||||
|
||||
if (null != bootstrapBasePath) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP, bootstrapBasePath);
|
||||
tableConfig.setValue(HoodieTableConfig.BOOTSTRAP_BASE_PATH, bootstrapBasePath);
|
||||
}
|
||||
|
||||
if (null != preCombineField) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PRECOMBINE_FIELD_PROP, preCombineField);
|
||||
tableConfig.setValue(HoodieTableConfig.PRECOMBINE_FIELD, preCombineField);
|
||||
}
|
||||
|
||||
if (null != partitionFields) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_PARTITION_FIELDS_PROP, partitionFields);
|
||||
tableConfig.setValue(HoodieTableConfig.PARTITION_FIELDS, partitionFields);
|
||||
}
|
||||
if (null != recordKeyFields) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_RECORDKEY_FIELDS, recordKeyFields);
|
||||
tableConfig.setValue(HoodieTableConfig.RECORDKEY_FIELDS, recordKeyFields);
|
||||
}
|
||||
if (null != populateMetaFields) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
|
||||
tableConfig.setValue(HoodieTableConfig.POPULATE_META_FIELDS, Boolean.toString(populateMetaFields));
|
||||
}
|
||||
if (null != keyGeneratorClassProp) {
|
||||
tableConfig.setValue(HoodieTableConfig.HOODIE_TABLE_KEY_GENERATOR_CLASS, keyGeneratorClassProp);
|
||||
tableConfig.setValue(HoodieTableConfig.KEY_GENERATOR_CLASS_NAME, keyGeneratorClassProp);
|
||||
}
|
||||
return tableConfig.getProps();
|
||||
}
|
||||
|
||||
@@ -40,7 +40,7 @@ import java.util.stream.Collectors;
|
||||
public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
|
||||
// Property Names
|
||||
public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_VIEW_STORAGE_TYPE = ConfigProperty
|
||||
public static final ConfigProperty<FileSystemViewStorageType> VIEW_TYPE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("File system view provides APIs for viewing the files on the underlying lake storage, "
|
||||
@@ -48,69 +48,69 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
+ Arrays.stream(FileSystemViewStorageType.values()).map(Enum::name).collect(Collectors.joining(","))
|
||||
+ " which provide different trade offs for memory usage and API request performance.");
|
||||
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = ConfigProperty
|
||||
public static final ConfigProperty<String> INCREMENTAL_TIMELINE_SYNC_ENABLE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.incr.timeline.sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("Controls whether or not, the file system view is incrementally updated as "
|
||||
+ "new actions are performed on the timeline.");
|
||||
|
||||
public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = ConfigProperty
|
||||
public static final ConfigProperty<FileSystemViewStorageType> SECONDARY_VIEW_TYPE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.secondary.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("Specifies the secondary form of storage for file system view, if the primary (e.g timeline server) "
|
||||
+ " is unavailable.");
|
||||
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_REMOTE_HOST = ConfigProperty
|
||||
public static final ConfigProperty<String> REMOTE_HOST_NAME = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.host")
|
||||
.defaultValue("localhost")
|
||||
.withDocumentation("We expect this to be rarely hand configured.");
|
||||
|
||||
public static final ConfigProperty<Integer> FILESYSTEM_VIEW_REMOTE_PORT = ConfigProperty
|
||||
public static final ConfigProperty<Integer> REMOTE_PORT_NUM = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.port")
|
||||
.defaultValue(26754)
|
||||
.withDocumentation("Port to serve file system view queries, when remote. We expect this to be rarely hand configured.");
|
||||
|
||||
public static final ConfigProperty<String> FILESYSTEM_VIEW_SPILLABLE_DIR = ConfigProperty
|
||||
public static final ConfigProperty<String> SPILLABLE_DIR = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.dir")
|
||||
.defaultValue("/tmp/")
|
||||
.withDocumentation("Path on local storage to use, when file system view is held in a spillable map.");
|
||||
|
||||
public static final ConfigProperty<Long> FILESYSTEM_VIEW_SPILLABLE_MEM = ConfigProperty
|
||||
public static final ConfigProperty<Long> SPILLABLE_MEMORY = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.mem")
|
||||
.defaultValue(100 * 1024 * 1024L) // 100 MB
|
||||
.withDocumentation("Amount of memory to be used for holding file system view, before spilling to disk.");
|
||||
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = ConfigProperty
|
||||
public static final ConfigProperty<Double> SPILLABLE_COMPACTION_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.compaction.mem.fraction")
|
||||
.defaultValue(0.8)
|
||||
.withDocumentation("Fraction of the file system view memory, to be used for holding compaction related metadata.");
|
||||
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = ConfigProperty
|
||||
public static final ConfigProperty<Double> BOOTSTRAP_BASE_FILE_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
|
||||
.defaultValue(0.05)
|
||||
.withDocumentation("Fraction of the file system view memory, to be used for holding mapping to bootstrap base files.");
|
||||
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = ConfigProperty
|
||||
public static final ConfigProperty<Double> SPILLABLE_REPLACED_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.replaced.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("Fraction of the file system view memory, to be used for holding replace commit related metadata.");
|
||||
|
||||
public static final ConfigProperty<Double> FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = ConfigProperty
|
||||
public static final ConfigProperty<Double> SPILLABLE_CLUSTERING_MEM_FRACTION = ConfigProperty
|
||||
.key("hoodie.filesystem.view.spillable.clustering.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("Fraction of the file system view memory, to be used for holding clustering related metadata.");
|
||||
|
||||
public static final ConfigProperty<String> ROCKSDB_BASE_PATH_PROP = ConfigProperty
|
||||
public static final ConfigProperty<String> ROCKSDB_BASE_PATH = ConfigProperty
|
||||
.key("hoodie.filesystem.view.rocksdb.base.path")
|
||||
.defaultValue("/tmp/hoodie_timeline_rocksdb")
|
||||
.withDocumentation("Path on local storage to use, when storing file system view in embedded kv store/rocksdb.");
|
||||
|
||||
public static final ConfigProperty<Integer> FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
|
||||
public static final ConfigProperty<Integer> REMOTE_TIMEOUT_SECS = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.timeout.secs")
|
||||
.defaultValue(5 * 60) // 5 min
|
||||
.withDocumentation("Timeout in seconds, to wait for API requests against a remote file system view. e.g timeline server.");
|
||||
|
||||
public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_HANDLER_ENABLE = ConfigProperty
|
||||
public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_ENABLE = ConfigProperty
|
||||
.key("hoodie.filesystem.remote.backup.view.enable")
|
||||
.defaultValue("true") // Need to be disabled only for tests.
|
||||
.withDocumentation("Config to control whether backup needs to be configured if clients were not able to reach"
|
||||
@@ -125,70 +125,70 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
return FileSystemViewStorageType.valueOf(getString(VIEW_TYPE));
|
||||
}
|
||||
|
||||
public boolean isIncrementalTimelineSyncEnabled() {
|
||||
return getBoolean(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE);
|
||||
return getBoolean(INCREMENTAL_TIMELINE_SYNC_ENABLE);
|
||||
}
|
||||
|
||||
public String getRemoteViewServerHost() {
|
||||
return getString(FILESYSTEM_VIEW_REMOTE_HOST);
|
||||
return getString(REMOTE_HOST_NAME);
|
||||
}
|
||||
|
||||
public Integer getRemoteViewServerPort() {
|
||||
return getInt(FILESYSTEM_VIEW_REMOTE_PORT);
|
||||
return getInt(REMOTE_PORT_NUM);
|
||||
}
|
||||
|
||||
public Integer getRemoteTimelineClientTimeoutSecs() {
|
||||
return getInt(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
|
||||
return getInt(REMOTE_TIMEOUT_SECS);
|
||||
}
|
||||
|
||||
public long getMaxMemoryForFileGroupMap() {
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
long totalMemory = getLong(SPILLABLE_MEMORY);
|
||||
return totalMemory - getMaxMemoryForPendingCompaction() - getMaxMemoryForBootstrapBaseFile();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForPendingCompaction() {
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION))
|
||||
long totalMemory = getLong(SPILLABLE_MEMORY);
|
||||
return new Double(totalMemory * getDouble(SPILLABLE_COMPACTION_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForBootstrapBaseFile() {
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
long totalMemory = getLong(SPILLABLE_MEMORY);
|
||||
long reservedForExternalDataFile =
|
||||
new Double(totalMemory * getDouble(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION))
|
||||
new Double(totalMemory * getDouble(BOOTSTRAP_BASE_FILE_MEM_FRACTION))
|
||||
.longValue();
|
||||
return reservedForExternalDataFile;
|
||||
}
|
||||
|
||||
public long getMaxMemoryForReplacedFileGroups() {
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION))
|
||||
long totalMemory = getLong(SPILLABLE_MEMORY);
|
||||
return new Double(totalMemory * getDouble(SPILLABLE_REPLACED_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public long getMaxMemoryForPendingClusteringFileGroups() {
|
||||
long totalMemory = getLong(FILESYSTEM_VIEW_SPILLABLE_MEM);
|
||||
return new Double(totalMemory * getDouble(FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION))
|
||||
long totalMemory = getLong(SPILLABLE_MEMORY);
|
||||
return new Double(totalMemory * getDouble(SPILLABLE_CLUSTERING_MEM_FRACTION))
|
||||
.longValue();
|
||||
}
|
||||
|
||||
public String getSpillableDir() {
|
||||
return getString(FILESYSTEM_VIEW_SPILLABLE_DIR);
|
||||
return getString(SPILLABLE_DIR);
|
||||
}
|
||||
|
||||
public FileSystemViewStorageType getSecondaryStorageType() {
|
||||
return FileSystemViewStorageType.valueOf(getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
return FileSystemViewStorageType.valueOf(getString(SECONDARY_VIEW_TYPE));
|
||||
}
|
||||
|
||||
public boolean shouldEnableBackupForRemoteFileSystemView() {
|
||||
return getBoolean(REMOTE_BACKUP_VIEW_HANDLER_ENABLE);
|
||||
return getBoolean(REMOTE_BACKUP_VIEW_ENABLE);
|
||||
}
|
||||
|
||||
public String getRocksdbBasePath() {
|
||||
return getString(ROCKSDB_BASE_PATH_PROP);
|
||||
return getString(ROCKSDB_BASE_PATH);
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -211,73 +211,203 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Builder withStorageType(FileSystemViewStorageType storageType) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_STORAGE_TYPE, storageType.name());
|
||||
fileSystemViewStorageConfig.setValue(VIEW_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withSecondaryStorageType(FileSystemViewStorageType storageType) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE, storageType.name());
|
||||
fileSystemViewStorageConfig.setValue(SECONDARY_VIEW_TYPE, storageType.name());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withIncrementalTimelineSync(boolean enableIncrTimelineSync) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE, Boolean.toString(enableIncrTimelineSync));
|
||||
fileSystemViewStorageConfig.setValue(INCREMENTAL_TIMELINE_SYNC_ENABLE, Boolean.toString(enableIncrTimelineSync));
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerHost(String remoteServerHost) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_HOST, remoteServerHost);
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_HOST_NAME, remoteServerHost);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteServerPort(Integer remoteServerPort) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_REMOTE_PORT, remoteServerPort.toString());
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_PORT_NUM, remoteServerPort.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMaxMemoryForView(Long maxMemoryForView) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_MEM, maxMemoryForView.toString());
|
||||
fileSystemViewStorageConfig.setValue(SPILLABLE_MEMORY, maxMemoryForView.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRemoteTimelineClientTimeoutSecs(Long timelineClientTimeoutSecs) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemFractionForPendingCompaction(Double memFractionForPendingCompaction) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
|
||||
fileSystemViewStorageConfig.setValue(SPILLABLE_COMPACTION_MEM_FRACTION, memFractionForPendingCompaction.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withMemFractionForExternalDataFile(Double memFractionForExternalDataFile) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, memFractionForExternalDataFile.toString());
|
||||
fileSystemViewStorageConfig.setValue(BOOTSTRAP_BASE_FILE_MEM_FRACTION, memFractionForExternalDataFile.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withBaseStoreDir(String baseStorePath) {
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_VIEW_SPILLABLE_DIR, baseStorePath);
|
||||
fileSystemViewStorageConfig.setValue(SPILLABLE_DIR, baseStorePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withRocksDBPath(String basePath) {
|
||||
fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH_PROP, basePath);
|
||||
fileSystemViewStorageConfig.setValue(ROCKSDB_BASE_PATH, basePath);
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder withEnableBackupForRemoteFileSystemView(boolean enable) {
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_HANDLER_ENABLE, Boolean.toString(enable));
|
||||
fileSystemViewStorageConfig.setValue(REMOTE_BACKUP_VIEW_ENABLE, Boolean.toString(enable));
|
||||
return this;
|
||||
}
|
||||
|
||||
public FileSystemViewStorageConfig build() {
|
||||
fileSystemViewStorageConfig.setDefaults(FileSystemViewStorageConfig.class.getName());
|
||||
// Validations
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_VIEW_STORAGE_TYPE));
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE));
|
||||
ValidationUtils.checkArgument(fileSystemViewStorageConfig.getInt(FILESYSTEM_VIEW_REMOTE_PORT) > 0);
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(VIEW_TYPE));
|
||||
FileSystemViewStorageType.valueOf(fileSystemViewStorageConfig.getString(SECONDARY_VIEW_TYPE));
|
||||
ValidationUtils.checkArgument(fileSystemViewStorageConfig.getInt(REMOTE_PORT_NUM) > 0);
|
||||
return fileSystemViewStorageConfig;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #VIEW_TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_STORAGE_TYPE = VIEW_TYPE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #VIEW_TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = VIEW_TYPE.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #INCREMENTAL_TIMELINE_SYNC_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = INCREMENTAL_TIMELINE_SYNC_ENABLE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #INCREMENTAL_TIMELINE_SYNC_ENABLE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_FILESYSTEM_VIEW_INCREMENTAL_SYNC_MODE = INCREMENTAL_TIMELINE_SYNC_ENABLE.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #SECONDARY_VIEW_TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_SECONDARY_VIEW_STORAGE_TYPE = SECONDARY_VIEW_TYPE.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SECONDARY_VIEW_TYPE} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = SECONDARY_VIEW_TYPE.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_HOST_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_HOST = REMOTE_HOST_NAME.key();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_HOST_NAME} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFUALT_REMOTE_VIEW_SERVER_HOST = REMOTE_HOST_NAME.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_PORT_NUM} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_REMOTE_PORT = REMOTE_PORT_NUM.key();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_PORT_NUM} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final Integer DEFAULT_REMOTE_VIEW_SERVER_PORT = REMOTE_PORT_NUM.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_DIR} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_DIR = SPILLABLE_DIR.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_DIR} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_VIEW_SPILLABLE_DIR = SPILLABLE_DIR.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_MEMORY} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_SPILLABLE_MEM = SPILLABLE_MEMORY.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_MEMORY} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = SPILLABLE_MEMORY.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_COMPACTION_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION = SPILLABLE_COMPACTION_MEM_FRACTION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_COMPACTION_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = SPILLABLE_COMPACTION_MEM_FRACTION.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #BOOTSTRAP_BASE_FILE_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = BOOTSTRAP_BASE_FILE_MEM_FRACTION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_REPLACED_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = SPILLABLE_REPLACED_MEM_FRACTION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_REPLACED_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = SPILLABLE_REPLACED_MEM_FRACTION.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_CLUSTERING_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESYSTEM_VIEW_PENDING_CLUSTERING_MEM_FRACTION = SPILLABLE_CLUSTERING_MEM_FRACTION.key();
|
||||
/**
|
||||
* @deprecated Use {@link #SPILLABLE_CLUSTERING_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_CLUSTERING_FILEGROUPS = SPILLABLE_CLUSTERING_MEM_FRACTION.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #ROCKSDB_BASE_PATH} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final String ROCKSDB_BASE_PATH_PROP = ROCKSDB_BASE_PATH.key();
|
||||
/**
|
||||
* @deprecated Use {@link #ROCKSDB_BASE_PATH} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_ROCKSDB_BASE_PATH = ROCKSDB_BASE_PATH.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_TIMEOUT_SECS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = REMOTE_TIMEOUT_SECS.key();
|
||||
/**
|
||||
* @deprecated Use {@link #REMOTE_TIMEOUT_SECS} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final Integer DEFAULT_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = REMOTE_TIMEOUT_SECS.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #BOOTSTRAP_BASE_FILE_MEM_FRACTION} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = BOOTSTRAP_BASE_FILE_MEM_FRACTION.defaultValue();
|
||||
}
|
||||
|
||||
@@ -55,5 +55,36 @@ public class KeyGeneratorOptions extends HoodieConfig {
|
||||
.defaultValue("partitionpath")
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
|
||||
/**
|
||||
* @deprecated Use {@link #URL_ENCODE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String URL_ENCODE_PARTITIONING_OPT_KEY = URL_ENCODE_PARTITIONING.key();
|
||||
/**
|
||||
* @deprecated Use {@link #URL_ENCODE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_URL_ENCODE_PARTITIONING_OPT_VAL = URL_ENCODE_PARTITIONING.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #HIVE_STYLE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String HIVE_STYLE_PARTITIONING_OPT_KEY = HIVE_STYLE_PARTITIONING.key();
|
||||
/**
|
||||
* @deprecated Use {@link #HIVE_STYLE_PARTITIONING} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String DEFAULT_HIVE_STYLE_PARTITIONING_OPT_VAL = HIVE_STYLE_PARTITIONING.defaultValue();
|
||||
/**
|
||||
* @deprecated Use {@link #RECORDKEY_FIELD} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String RECORDKEY_FIELD_OPT_KEY = RECORDKEY_FIELD.key();
|
||||
/**
|
||||
* @deprecated Use {@link #PARTITIONPATH_FIELD} and its methods.
|
||||
*/
|
||||
@Deprecated
|
||||
public static final String PARTITIONPATH_FIELD_OPT_KEY = PARTITIONPATH_FIELD.key();
|
||||
}
|
||||
|
||||
|
||||
@@ -51,7 +51,7 @@ public class HoodieMetadataFileSystemView extends HoodieTableFileSystemView {
|
||||
HoodieMetadataConfig metadataConfig) {
|
||||
super(metaClient, visibleActiveTimeline);
|
||||
this.tableMetadata = HoodieTableMetadata.create(engineContext, metadataConfig, metaClient.getBasePath(),
|
||||
FileSystemViewStorageConfig.FILESYSTEM_VIEW_SPILLABLE_DIR.defaultValue());
|
||||
FileSystemViewStorageConfig.SPILLABLE_DIR.defaultValue());
|
||||
}
|
||||
|
||||
/**
|
||||
|
||||
@@ -34,6 +34,8 @@ import org.apache.hudi.common.util.collection.Pair;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
@@ -52,9 +54,6 @@ import java.util.concurrent.Future;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.IntStream;
|
||||
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertNotNull;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
@@ -92,7 +91,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
|
||||
@Test
|
||||
public void testNoOpBootstrapIndex() throws IOException {
|
||||
Properties props = metaClient.getTableConfig().getProps();
|
||||
props.put(HoodieTableConfig.HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP.key(), "false");
|
||||
props.put(HoodieTableConfig.BOOTSTRAP_INDEX_ENABLE.key(), "false");
|
||||
Properties properties = new Properties();
|
||||
properties.putAll(props);
|
||||
HoodieTableConfig.createHoodieProperties(metaClient.getFs(), new Path(metaClient.getMetaPath()), properties);
|
||||
@@ -185,7 +184,7 @@ public class TestBootstrapIndex extends HoodieCommonTestHarness {
|
||||
return Arrays.stream(partitions).map(partition -> {
|
||||
return Pair.of(partition, IntStream.range(0, numEntriesPerPartition).mapToObj(idx -> {
|
||||
String hudiFileId = UUID.randomUUID().toString();
|
||||
String sourceFileName = idx + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
|
||||
String sourceFileName = idx + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
|
||||
HoodieFileStatus sourceFileStatus = HoodieFileStatus.newBuilder()
|
||||
.setPath(HoodiePath.newBuilder().setUri(sourceBasePath + "/" + partition + "/" + sourceFileName).build())
|
||||
.setLength(256 * 1024 * 1024L)
|
||||
|
||||
@@ -18,14 +18,15 @@
|
||||
|
||||
package org.apache.hudi.common.fs;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodieLogFile;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
import org.apache.hudi.common.testutils.HoodieCommonTestHarness;
|
||||
import org.apache.hudi.common.testutils.HoodieTestUtils;
|
||||
import org.apache.hudi.exception.HoodieException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.Rule;
|
||||
import org.junit.contrib.java.lang.system.EnvironmentVariables;
|
||||
import org.junit.jupiter.api.BeforeEach;
|
||||
@@ -57,7 +58,7 @@ public class TestFSUtils extends HoodieCommonTestHarness {
|
||||
private final long minCleanToKeep = 10;
|
||||
|
||||
private static String TEST_WRITE_TOKEN = "1-0-1";
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
|
||||
|
||||
@Rule
|
||||
public final EnvironmentVariables environmentVariables = new EnvironmentVariables();
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.common.model;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
|
||||
import org.junit.jupiter.api.Test;
|
||||
|
||||
import java.util.ArrayList;
|
||||
@@ -34,7 +35,7 @@ public class TestHoodieDeltaWriteStat {
|
||||
@Test
|
||||
public void testBaseFileAndLogFiles() {
|
||||
HoodieDeltaWriteStat writeStat = new HoodieDeltaWriteStat();
|
||||
String baseFile = "file1" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
|
||||
String baseFile = "file1" + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
|
||||
String logFile1 = ".log1.log";
|
||||
String logFile2 = ".log2.log";
|
||||
|
||||
|
||||
@@ -68,7 +68,7 @@ public class TestPriorityBasedFileSystemView {
|
||||
fsView = new PriorityBasedFileSystemView(primary, secondary);
|
||||
testBaseFileStream = Stream.of(new HoodieBaseFile("test"));
|
||||
testFileSliceStream = Stream.of(new FileSlice("2020-01-01", "20:20",
|
||||
"file0001" + HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension()));
|
||||
"file0001" + HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension()));
|
||||
}
|
||||
|
||||
private void resetMocks() {
|
||||
|
||||
@@ -51,16 +51,16 @@ import java.time.Instant;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCompactionPlan;
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanMetadata;
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCleanerPlan;
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCompactionPlan;
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRequestedReplaceMetadata;
|
||||
import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata;
|
||||
|
||||
public class FileCreateUtils {
|
||||
|
||||
private static final String WRITE_TOKEN = "1-0-1";
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
|
||||
|
||||
public static String baseFileName(String instantTime, String fileId) {
|
||||
return baseFileName(instantTime, fileId, BASE_FILE_EXTENSION);
|
||||
|
||||
@@ -397,7 +397,7 @@ public class HoodieTestTable {
|
||||
}
|
||||
|
||||
public FileStatus[] listAllBaseFiles() throws IOException {
|
||||
return listAllBaseFiles(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension());
|
||||
return listAllBaseFiles(HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension());
|
||||
}
|
||||
|
||||
public FileStatus[] listAllBaseFiles(String fileExtension) throws IOException {
|
||||
|
||||
@@ -65,7 +65,7 @@ public class HoodieTestUtils {
|
||||
|
||||
public static HoodieTableMetaClient init(String basePath, HoodieTableType tableType, String bootstrapBasePath) throws IOException {
|
||||
Properties props = new Properties();
|
||||
props.setProperty(HoodieTableConfig.HOODIE_BOOTSTRAP_BASE_PATH_PROP.key(), bootstrapBasePath);
|
||||
props.setProperty(HoodieTableConfig.BOOTSTRAP_BASE_PATH.key(), bootstrapBasePath);
|
||||
return init(getDefaultHadoopConf(), basePath, tableType, props);
|
||||
}
|
||||
|
||||
@@ -86,7 +86,7 @@ public class HoodieTestUtils {
|
||||
String tableName)
|
||||
throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_TABLE_NAME_PROP.key(), tableName);
|
||||
properties.setProperty(HoodieTableConfig.NAME.key(), tableName);
|
||||
return init(hadoopConf, basePath, tableType, properties);
|
||||
}
|
||||
|
||||
@@ -94,7 +94,7 @@ public class HoodieTestUtils {
|
||||
HoodieFileFormat baseFileFormat)
|
||||
throws IOException {
|
||||
Properties properties = new Properties();
|
||||
properties.setProperty(HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.key(), baseFileFormat.toString());
|
||||
properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), baseFileFormat.toString());
|
||||
return init(hadoopConf, basePath, tableType, properties);
|
||||
}
|
||||
|
||||
|
||||
@@ -17,12 +17,13 @@
|
||||
|
||||
package org.apache.hudi.common.util;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.common.model.HoodiePartitionMetadata;
|
||||
import org.apache.hudi.common.table.HoodieTableConfig;
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.jupiter.api.BeforeAll;
|
||||
import org.junit.jupiter.api.Test;
|
||||
import org.junit.jupiter.api.io.TempDir;
|
||||
@@ -37,7 +38,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals;
|
||||
import static org.junit.jupiter.api.Assertions.assertTrue;
|
||||
|
||||
public final class TestTablePathUtils {
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.HOODIE_BASE_FILE_FORMAT_PROP.defaultValue().getFileExtension();
|
||||
private static final String BASE_FILE_EXTENSION = HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().getFileExtension();
|
||||
|
||||
@TempDir
|
||||
static File tempDir;
|
||||
|
||||
Reference in New Issue
Block a user