[HUDI-2149] Ensure and Audit docs for every configuration class in the codebase (#3272)
- Added docs when missing - Rewrote, reworded as needed - Made couple more classes extend HoodieConfig
This commit is contained in:
@@ -37,14 +37,14 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.key(METADATA_PREFIX + ".enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Enable the internal Metadata Table which stores table level file listings");
|
||||
.withDocumentation("Enable the internal metadata table which serves table metadata like level file listings");
|
||||
|
||||
// Validate contents of Metadata Table on each access against the actual filesystem
|
||||
public static final ConfigProperty<Boolean> METADATA_VALIDATE_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".validate")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Validate contents of Metadata Table on each access against the actual listings from DFS");
|
||||
.withDocumentation("Validate contents of metadata table on each access; e.g against the actual listings from lake storage");
|
||||
|
||||
public static final boolean DEFAULT_METADATA_ENABLE_FOR_READERS = false;
|
||||
|
||||
@@ -53,14 +53,14 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.key(METADATA_PREFIX + ".metrics.enable")
|
||||
.defaultValue(false)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Enable publishing of metrics around metadata table.");
|
||||
|
||||
// Parallelism for inserts
|
||||
public static final ConfigProperty<Integer> METADATA_INSERT_PARALLELISM_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".insert.parallelism")
|
||||
.defaultValue(1)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Parallelism to use when writing to the metadata table");
|
||||
.withDocumentation("Parallelism to use when inserting to the metadata table");
|
||||
|
||||
// Async clean
|
||||
public static final ConfigProperty<Boolean> METADATA_ASYNC_CLEAN_PROP = ConfigProperty
|
||||
@@ -81,32 +81,32 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.key(METADATA_PREFIX + ".keep.min.commits")
|
||||
.defaultValue(20)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline");
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline.");
|
||||
|
||||
public static final ConfigProperty<Integer> MAX_COMMITS_TO_KEEP_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".keep.max.commits")
|
||||
.defaultValue(30)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline");
|
||||
.withDocumentation("Controls the archival of the metadata table’s timeline.");
|
||||
|
||||
// Cleaner commits retained
|
||||
public static final ConfigProperty<Integer> CLEANER_COMMITS_RETAINED_PROP = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".cleaner.commits.retained")
|
||||
.defaultValue(3)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Controls retention/history for metadata table.");
|
||||
|
||||
// Regex to filter out matching directories during bootstrap
|
||||
public static final ConfigProperty<String> DIRECTORY_FILTER_REGEX = ConfigProperty
|
||||
.key(METADATA_PREFIX + ".dir.filter.regex")
|
||||
.defaultValue("")
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.assume.date.partitioning")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.7.0")
|
||||
.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");
|
||||
|
||||
@@ -114,7 +114,7 @@ public final class HoodieMetadataConfig extends HoodieConfig {
|
||||
.key("hoodie.file.listing.parallelism")
|
||||
.defaultValue(1500)
|
||||
.sinceVersion("0.7.0")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Parallelism to use, when listing the table on lake storage.");
|
||||
|
||||
private HoodieMetadataConfig() {
|
||||
super();
|
||||
|
||||
@@ -31,12 +31,11 @@ import java.util.Properties;
|
||||
*/
|
||||
public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
|
||||
// time between successive attempts to ensure written data's metadata is consistent on storage
|
||||
@Deprecated
|
||||
public static final ConfigProperty<String> CONSISTENCY_CHECK_ENABLED_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.enabled")
|
||||
.defaultValue("false")
|
||||
.sinceVersion("0.5.0")
|
||||
.deprecatedAfter("0.7.0")
|
||||
.withDocumentation("Enabled to handle S3 eventual consistency issue. This property is no longer required "
|
||||
+ "since S3 is now strongly consistent. Will be removed in the future releases.");
|
||||
|
||||
@@ -44,35 +43,37 @@ public class ConsistencyGuardConfig extends HoodieConfig {
|
||||
.key("hoodie.consistency.check.initial_interval_ms")
|
||||
.defaultValue(400L)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
.deprecatedAfter("0.7.0")
|
||||
.withDocumentation("Amount of time (in ms) to wait, before checking for consistency after an operation on storage.");
|
||||
|
||||
// max interval time
|
||||
public static final ConfigProperty<Long> MAX_CONSISTENCY_CHECK_INTERVAL_MS_PROP = ConfigProperty
|
||||
.key("hoodie.consistency.check.max_interval_ms")
|
||||
.defaultValue(20000L)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
.deprecatedAfter("0.7.0")
|
||||
.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
|
||||
.key("hoodie.consistency.check.max_checks")
|
||||
.defaultValue(6)
|
||||
.sinceVersion("0.5.0")
|
||||
.withDocumentation("");
|
||||
.deprecatedAfter("0.7.0")
|
||||
.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
|
||||
.key("hoodie.optimistic.consistency.guard.sleep_time_ms")
|
||||
.defaultValue(500L)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("_hoodie.optimistic.consistency.guard.enable")
|
||||
.defaultValue(true)
|
||||
.sinceVersion("0.6.0")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Enable consistency guard, which optimistically assumes consistency is achieved after a certain time period.");
|
||||
|
||||
private ConsistencyGuardConfig() {
|
||||
super();
|
||||
|
||||
@@ -23,7 +23,8 @@ import org.apache.hadoop.fs.Path;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Default Consistency guard that does nothing. Used for HDFS deployments
|
||||
* Default Consistency guard that does nothing. Used for lake storage which provided read-after-write
|
||||
* guarantees.
|
||||
*/
|
||||
public class NoOpConsistencyGuard implements ConsistencyGuard {
|
||||
|
||||
|
||||
@@ -72,66 +72,69 @@ public class HoodieTableConfig extends HoodieConfig implements Serializable {
|
||||
public static final ConfigProperty<HoodieTableVersion> HOODIE_TABLE_VERSION_PROP = ConfigProperty
|
||||
.key("hoodie.table.version")
|
||||
.defaultValue(HoodieTableVersion.ZERO)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.table.precombine.field")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Field used in preCombining before actual write. When two records have the same key value, "
|
||||
+ "we will pick the one with the largest value for the precombine field, determined by Object.compareTo(..)");
|
||||
.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_COLUMNS_PROP = ConfigProperty
|
||||
.key("hoodie.table.partition.columns")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("Partition path field. Value to be used at the partitionPath component of HoodieKey. "
|
||||
+ "Actual value ontained by invoking .toString()");
|
||||
.withDocumentation("Columns 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
|
||||
.key("hoodie.table.recordkey.fields")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.table.create.schema")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Schema used when creating the table, for the first time.");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_BASE_FILE_FORMAT_PROP = ConfigProperty
|
||||
.key("hoodie.table.base.file.format")
|
||||
.defaultValue(HoodieFileFormat.PARQUET)
|
||||
.withAlternatives("hoodie.table.ro.file.format")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Base file format to store all the base file data.");
|
||||
|
||||
public static final ConfigProperty<HoodieFileFormat> HOODIE_LOG_FILE_FORMAT_PROP = ConfigProperty
|
||||
.key("hoodie.table.log.file.format")
|
||||
.defaultValue(HoodieFileFormat.HOODIE_LOG)
|
||||
.withAlternatives("hoodie.table.rt.file.format")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Log format used for the delta logs.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_TIMELINE_LAYOUT_VERSION_PROP = ConfigProperty
|
||||
.key("hoodie.timeline.layout.version")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Version of timeline used, by the table.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_PAYLOAD_CLASS_PROP = ConfigProperty
|
||||
.key("hoodie.compaction.payload.class")
|
||||
.defaultValue(OverwriteWithLatestAvroPayload.class.getName())
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.archivelog.folder")
|
||||
.defaultValue("archived")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("path under the meta folder, to store archived timeline instants at.");
|
||||
|
||||
public static final ConfigProperty<String> HOODIE_BOOTSTRAP_INDEX_ENABLE_PROP = ConfigProperty
|
||||
.key("hoodie.bootstrap.index.enable")
|
||||
.noDefaultValue()
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.bootstrap.index.class")
|
||||
.defaultValue(HFileBootstrapIndex.class.getName())
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.bootstrap.base.path")
|
||||
|
||||
@@ -34,7 +34,7 @@ public class TimelineLayoutVersion implements Serializable, Comparable<TimelineL
|
||||
public static final Integer CURR_VERSION = VERSION_1;
|
||||
public static final TimelineLayoutVersion CURR_LAYOUT_VERSION = new TimelineLayoutVersion(CURR_VERSION);
|
||||
|
||||
private Integer version;
|
||||
private final Integer version;
|
||||
|
||||
public TimelineLayoutVersion(Integer version) {
|
||||
ValidationUtils.checkArgument(version <= CURR_VERSION);
|
||||
|
||||
@@ -25,7 +25,9 @@ import org.apache.hudi.common.util.ValidationUtils;
|
||||
import java.io.File;
|
||||
import java.io.FileReader;
|
||||
import java.io.IOException;
|
||||
import java.util.Arrays;
|
||||
import java.util.Properties;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* File System View Storage Configurations.
|
||||
@@ -36,76 +38,78 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
public static final ConfigProperty<FileSystemViewStorageType> FILESYSTEM_VIEW_STORAGE_TYPE = ConfigProperty
|
||||
.key("hoodie.filesystem.view.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("");
|
||||
.withDocumentation("File system view provides APIs for viewing the files on the underlying lake storage, "
|
||||
+ " as file groups and file slices. This config controls how such a view is held. Options include "
|
||||
+ 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
|
||||
.key("hoodie.filesystem.view.incr.timeline.sync.enable")
|
||||
.defaultValue("false")
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.secondary.type")
|
||||
.defaultValue(FileSystemViewStorageType.MEMORY)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.remote.host")
|
||||
.defaultValue("localhost")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("We expect this to be rarely hand configured.");
|
||||
|
||||
public static final ConfigProperty<Integer> FILESYSTEM_VIEW_REMOTE_PORT = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.port")
|
||||
.defaultValue(26754)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.dir")
|
||||
.defaultValue("/tmp/view_map/")
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.mem")
|
||||
.defaultValue(100 * 1024 * 1024L) // 100 MB
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.compaction.mem.fraction")
|
||||
.defaultValue(0.8)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction")
|
||||
.defaultValue(0.05)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.replaced.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.spillable.clustering.mem.fraction")
|
||||
.defaultValue(0.01)
|
||||
.withDocumentation("");
|
||||
.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
|
||||
.key("hoodie.filesystem.view.rocksdb.base.path")
|
||||
.defaultValue("/tmp/hoodie_timeline_rocksdb")
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Path on local storage to use, when storing file system view in embedded kv store/rocksdb.");
|
||||
|
||||
public static final ConfigProperty<Integer> FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
|
||||
public static final ConfigProperty<Integer> FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = ConfigProperty
|
||||
.key("hoodie.filesystem.view.remote.timeout.secs")
|
||||
.defaultValue(5 * 60) // 5 min
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Timeout in seconds, to wait for API requests against a remote file system view. e.g timeline server.");
|
||||
|
||||
/**
|
||||
* Configs to control whether backup needs to be configured if clients were not able to reach
|
||||
* timeline service.
|
||||
*/
|
||||
public static final ConfigProperty<String> REMOTE_BACKUP_VIEW_HANDLER_ENABLE = ConfigProperty
|
||||
.key("hoodie.filesystem.remote.backup.view.enable")
|
||||
.defaultValue("true") // Need to be disabled only for tests.
|
||||
.withDocumentation("");
|
||||
.withDocumentation("Config to control whether backup needs to be configured if clients were not able to reach"
|
||||
+ " timeline service.");
|
||||
|
||||
public static FileSystemViewStorageConfig.Builder newBuilder() {
|
||||
return new Builder();
|
||||
@@ -132,7 +136,7 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Integer getRemoteTimelineClientTimeoutSecs() {
|
||||
return getInt(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
|
||||
return getInt(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS);
|
||||
}
|
||||
|
||||
public long getMaxMemoryForFileGroupMap() {
|
||||
@@ -232,7 +236,7 @@ public class FileSystemViewStorageConfig extends HoodieConfig {
|
||||
}
|
||||
|
||||
public Builder withRemoteTimelineClientTimeoutSecs(Long timelineClientTimeoutSecs) {
|
||||
fileSystemViewStorageConfig.setValue(FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
fileSystemViewStorageConfig.setValue(FILESYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS, timelineClientTimeoutSecs.toString());
|
||||
return this;
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user