All Configurations
This page covers the different ways of configuring your job to write/read Hudi tables. At a high level, you can control behaviour at few levels.
- Spark Datasource Configs: These configs control the Hudi Spark Datasource, providing ability to define keys/partitioning, pick out the write operation, specify how to merge records or choosing query type to read.
- Flink Sql Configs: These configs control the Hudi Flink SQL source/sink connectors, providing ability to define record keys, pick out the write operation, specify how to merge records, enable/disable asynchronous compaction or choosing query type to read.
- Write Client Configs: Internally, the Hudi datasource uses a RDD based HoodieWriteClient API to actually perform writes to storage. These configs provide deep control over lower level aspects like file sizing, compression, parallelism, compaction, write schema, cleaning etc. Although Hudi provides sane defaults, from time-time these configs may need to be tweaked to optimize for specific workloads.
- Metrics Configs: These set of configs are used to enable monitoring and reporting of keyHudi stats and metrics.
- Record Payload Config: This is the lowest level of customization offered by Hudi. Record payloads define how to produce new values to upsert based on incoming new record and stored old record. Hudi provides default implementations such as OverwriteWithLatestAvroPayload which simply update table with the latest/last-written record. This can be overridden to a custom class extending HoodieRecordPayload class, on both datasource and WriteClient levels.
- Kafka Connect Configs: These set of configs are used for Kafka Connect Sink Connector for writing Hudi Tables
- Amazon Web Services Configs: Please fill in the description for Config Group Name: Amazon Web Services Configs
Externalized Config File
Instead of directly passing configuration settings to every Hudi job, you can also centrally set them in a configuration
file hudi-defaults.conf
. By default, Hudi would load the configuration file under /etc/hudi/conf
directory. You can
specify a different configuration directory location by setting the HUDI_CONF_DIR
environment variable. This can be
useful for uniformly enforcing repeated configs (like Hive sync or write/index tuning), across your entire data lake.
Spark Datasource Configs
These configs control the Hudi Spark Datasource, providing ability to define keys/partitioning, pick out the write operation, specify how to merge records or choosing query type to read.
Read Options
Options useful for reading tables via read.format.option(...)
Config Class
: org.apache.hudi.DataSourceOptions.scala
hoodie.file.index.enable
Enables use of the spark file index implementation for Hudi, that speeds up listing of large tables.
Default Value: true (Optional)
Config Param: ENABLE_HOODIE_FILE_INDEX
Deprecated Version: 0.11.0
hoodie.datasource.read.paths
Comma separated list of file paths to read within a Hudi table.
Default Value: N/A (Required)
Config Param: READ_PATHS
hoodie.datasource.read.incr.filters
For use-cases like DeltaStreamer which reads from Hoodie Incremental table and applies opaque map functions, filters appearing late in the sequence of transformations cannot be automatically pushed down. This option allows setting filters directly on Hoodie Source.
Default Value: (Optional)
Config Param: PUSH_DOWN_INCR_FILTERS
hoodie.enable.data.skipping
Enables data-skipping allowing queries to leverage indexes to reduce the search space by skipping over files
Default Value: false (Optional)
Config Param: ENABLE_DATA_SKIPPING
Since Version: 0.10.0
as.of.instant
The query instant for time travel. Without specified this option, we query the latest snapshot.
Default Value: N/A (Required)
Config Param: TIME_TRAVEL_AS_OF_INSTANT
hoodie.datasource.read.schema.use.end.instanttime
Uses end instant schema when incrementally fetched data to. Default: users latest instant schema.
Default Value: false (Optional)
Config Param: INCREMENTAL_READ_SCHEMA_USE_END_INSTANTTIME
hoodie.datasource.read.incr.path.glob
For the use-cases like users only want to incremental pull from certain partitions instead of the full table. This option allows using glob pattern to directly filter on path.
Default Value: (Optional)
Config Param: INCR_PATH_GLOB
hoodie.datasource.read.end.instanttime
Instant time to limit incrementally fetched data to. New data written with an instant_time <= END_INSTANTTIME are fetched out.
Default Value: N/A (Required)
Config Param: END_INSTANTTIME
hoodie.datasource.write.precombine.field
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(..)
Default Value: ts (Optional)
Config Param: READ_PRE_COMBINE_FIELD
hoodie.datasource.merge.type
For Snapshot query on merge on read table, control whether we invoke the record payload implementation to merge (payload_combine) or skip merging altogetherskip_merge
Default Value: payload_combine (Optional)
Config Param: REALTIME_MERGE
hoodie.datasource.read.extract.partition.values.from.path
When set to true, values for partition columns (partition values) will be extracted from physical partition path (default Spark behavior). When set to false partition values will be read from the data file (in Hudi partition columns are persisted by default). This config is a fallback allowing to preserve existing behavior, and should not be used otherwise.
Default Value: false (Optional)
Config Param: EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH
Since Version: 0.11.0
hoodie.schema.on.read.enable
Enables support for Schema Evolution feature
Default Value: false (Optional)
Config Param: SCHEMA_EVOLUTION_ENABLED
hoodie.datasource.read.begin.instanttime
Instant time to start incrementally pulling data from. The instanttime here need not necessarily correspond to an instant on the timeline. New data written with an instant_time > BEGIN_INSTANTTIME are fetched out. For e.g: ‘20170901080000’ will get all new data written after Sep 1, 2017 08:00AM.
Default Value: N/A (Required)
Config Param: BEGIN_INSTANTTIME
hoodie.datasource.read.incr.fallback.fulltablescan.enable
When doing an incremental query whether we should fall back to full table scans if file does not exist.
Default Value: false (Optional)
Config Param: INCREMENTAL_FALLBACK_TO_FULL_TABLE_SCAN_FOR_NON_EXISTING_FILES
hoodie.datasource.query.type
Whether data needs to be read, in incremental mode (new data since an instantTime) (or) Read Optimized mode (obtain latest view, based on base files) (or) Snapshot mode (obtain latest view, by merging base and (if any) log files)
Default Value: snapshot (Optional)
Config Param: QUERY_TYPE
Write Options
You can pass down any of the WriteClient level configs directly using options()
or option(k,v)
methods.
inputDF.write()
.format("org.apache.hudi")
.options(clientOpts) // any of the Hudi client opts can be passed in as well
.option(DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY(), "_row_key")
.option(DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY(), "partition")
.option(DataSourceWriteOptions.PRECOMBINE_FIELD_OPT_KEY(), "timestamp")
.option(HoodieWriteConfig.TABLE_NAME, tableName)
.mode(SaveMode.Append)
.save(basePath);
Options useful for writing tables via write.format.option(...)
Config Class
: org.apache.hudi.DataSourceOptions.scala
hoodie.clustering.async.enabled
Enable running of clustering service, asynchronously as inserts happen on the table.
Default Value: false (Optional)
Config Param: ASYNC_CLUSTERING_ENABLE
Since Version: 0.7.0
hoodie.datasource.write.operation
Whether to do upsert, insert or bulkinsert for the write operation. Use bulkinsert to load new data into a table, and there on use upsert/insert. bulk insert uses a disk based write path to scale to load large inputs without need to cache it.
Default Value: upsert (Optional)
Config Param: OPERATION
hoodie.datasource.write.reconcile.schema
When a new batch of write has records with old schema, but latest table schema got evolved, this config will upgrade the records to leverage latest table schema(default values will be injected to missing fields). If not, the write batch would fail.
Default Value: false (Optional)
Config Param: RECONCILE_SCHEMA
hoodie.datasource.write.recordkey.field
Record key field. Value to be used as the
recordKey
component ofHoodieKey
. Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using the dot notation eg:a.b.c
Default Value: uuid (Optional)
Config Param: RECORDKEY_FIELD
hoodie.datasource.hive_sync.skip_ro_suffix
Skip the _ro suffix for Read optimized table, when registering
Default Value: false (Optional)
Config Param: HIVE_SKIP_RO_SUFFIX_FOR_READ_OPTIMIZED_TABLE
hoodie.datasource.write.partitionpath.urlencode
Should we url encode the partition path value, before creating the folder structure.
Default Value: false (Optional)
Config Param: URL_ENCODE_PARTITIONING
hoodie.datasource.hive_sync.partition_extractor_class
Class which implements PartitionValueExtractor to extract the partition values, default 'org.apache.hudi.hive.MultiPartKeysValueExtractor'.
Default Value: org.apache.hudi.hive.MultiPartKeysValueExtractor (Optional)
Config Param: HIVE_PARTITION_EXTRACTOR_CLASS
hoodie.datasource.hive_sync.serde_properties
Serde properties to hive table.
Default Value: N/A (Required)
Config Param: HIVE_TABLE_SERDE_PROPERTIES
hoodie.datasource.hive_sync.sync_comment
Whether to sync the table column comments while syncing the table.
Default Value: false (Optional)
Config Param: HIVE_SYNC_COMMENT
hoodie.datasource.hive_sync.password
hive password to use
Default Value: hive (Optional)
Config Param: HIVE_PASS
hoodie.datasource.write.keygenerator.consistent.logical.timestamp.enabled
When set to true, consistent value will be generated for a logical timestamp type column, like timestamp-millis and timestamp-micros, irrespective of whether row-writer is enabled. Disabled by default so as not to break the pipeline that deploy either fully row-writer path or non row-writer path. For example, if it is kept disabled then record key of timestamp type with value
2016-12-29 09:54:00
will be written as timestamp2016-12-29 09:54:00.0
in row-writer path, while it will be written as long value1483023240000000
in non row-writer path. If enabled, then the timestamp value will be written in both the cases.
Default Value: false (Optional)
Config Param: KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED
hoodie.datasource.hive_sync.support_timestamp
‘INT64’ with original type TIMESTAMP_MICROS is converted to hive ‘timestamp’ type. Disabled by default for backward compatibility.
Default Value: false (Optional)
Config Param: HIVE_SUPPORT_TIMESTAMP_TYPE
hoodie.datasource.hive_sync.create_managed_table
Whether to sync the table as managed table.
Default Value: false (Optional)
Config Param: HIVE_CREATE_MANAGED_TABLE
hoodie.clustering.inline
Turn on inline clustering - clustering will be run after each write operation is complete
Default Value: false (Optional)
Config Param: INLINE_CLUSTERING_ENABLE
Since Version: 0.7.0
hoodie.datasource.compaction.async.enable
Controls whether async compaction should be turned on for MOR table writing.
Default Value: true (Optional)
Config Param: ASYNC_COMPACT_ENABLE
hoodie.datasource.meta.sync.enable
Enable Syncing the Hudi Table with an external meta store or data catalog.
Default Value: false (Optional)
Config Param: META_SYNC_ENABLED
hoodie.datasource.write.streaming.ignore.failed.batch
Config to indicate whether to ignore any non exception error (e.g. writestatus error) within a streaming microbatch
Default Value: true (Optional)
Config Param: STREAMING_IGNORE_FAILED_BATCH
hoodie.datasource.write.precombine.field
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(..)
Default Value: ts (Optional)
Config Param: PRECOMBINE_FIELD
hoodie.datasource.hive_sync.username
hive user name to use
Default Value: hive (Optional)
Config Param: HIVE_USER
hoodie.datasource.write.partitionpath.field
Partition path field. Value to be used at the partitionPath component of HoodieKey. Actual value ontained by invoking .toString()
Default Value: N/A (Required)
Config Param: PARTITIONPATH_FIELD
hoodie.datasource.write.streaming.retry.count
Config to indicate how many times streaming job should retry for a failed micro batch.
Default Value: 3 (Optional)
Config Param: STREAMING_RETRY_CNT
hoodie.datasource.hive_sync.partition_fields
Field in the table to use for determining hive partition columns.
Default Value: (Optional)
Config Param: HIVE_PARTITION_FIELDS
hoodie.datasource.hive_sync.sync_as_datasource
Default Value: true (Optional)
Config Param: HIVE_SYNC_AS_DATA_SOURCE_TABLE
hoodie.sql.insert.mode
Insert mode when insert data to pk-table. The optional modes are: upsert, strict and non-strict.For upsert mode, insert statement do the upsert operation for the pk-table which will update the duplicate record.For strict mode, insert statement will keep the primary key uniqueness constraint which do not allow duplicate record.While for non-strict mode, hudi just do the insert operation for the pk-table.
Default Value: upsert (Optional)
Config Param: SQL_INSERT_MODE
hoodie.datasource.hive_sync.use_jdbc
Use JDBC when hive synchronization is enabled
Default Value: true (Optional)
Config Param: HIVE_USE_JDBC
Deprecated Version: 0.9.0
hoodie.meta.sync.client.tool.class
Sync tool class name used to sync to metastore. Defaults to Hive.
Default Value: org.apache.hudi.hive.HiveSyncTool (Optional)
Config Param: META_SYNC_CLIENT_TOOL_CLASS_NAME
hoodie.datasource.write.keygenerator.class
Key generator class, that implements
org.apache.hudi.keygen.KeyGenerator
Default Value: org.apache.hudi.keygen.SimpleKeyGenerator (Optional)
Config Param: KEYGENERATOR_CLASS_NAME
hoodie.datasource.write.payload.class
Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective
Default Value: org.apache.hudi.common.model.OverwriteWithLatestAvroPayload (Optional)
Config Param: PAYLOAD_CLASS_NAME
hoodie.datasource.hive_sync.table_properties
Additional properties to store with table.
Default Value: N/A (Required)
Config Param: HIVE_TABLE_PROPERTIES
hoodie.datasource.hive_sync.jdbcurl
Hive metastore url
Default Value: jdbc:hive2://localhost:10000 (Optional)
Config Param: HIVE_URL
hoodie.datasource.hive_sync.batch_num
The number of partitions one batch when synchronous partitions to hive.
Default Value: 1000 (Optional)
Config Param: HIVE_BATCH_SYNC_PARTITION_NUM
hoodie.datasource.hive_sync.assume_date_partitioning
Assume partitioning is yyyy/MM/dd
Default Value: false (Optional)
Config Param: HIVE_ASSUME_DATE_PARTITION
hoodie.datasource.hive_sync.bucket_sync
Whether sync hive metastore bucket specification when using bucket index.The specification is 'CLUSTERED BY (trace_id) SORTED BY (trace_id ASC) INTO 65536 BUCKETS'
Default Value: false (Optional)
Config Param: HIVE_SYNC_BUCKET_SYNC
hoodie.datasource.hive_sync.auto_create_database
Auto create hive database if does not exists
Default Value: true (Optional)
Config Param: HIVE_AUTO_CREATE_DATABASE
hoodie.datasource.hive_sync.database
The name of the destination database that we should sync the hudi table to.
Default Value: default (Optional)
Config Param: HIVE_DATABASE
hoodie.datasource.write.streaming.retry.interval.ms
Config to indicate how long (by millisecond) before a retry should issued for failed microbatch
Default Value: 2000 (Optional)
Config Param: STREAMING_RETRY_INTERVAL_MS
hoodie.sql.bulk.insert.enable
When set to true, the sql insert statement will use bulk insert.
Default Value: false (Optional)
Config Param: SQL_ENABLE_BULK_INSERT
hoodie.datasource.write.commitmeta.key.prefix
Option keys beginning with this prefix, are automatically added to the commit/deltacommit metadata. This is useful to store checkpointing information, in a consistent way with the hudi timeline
Default Value: _ (Optional)
Config Param: COMMIT_METADATA_KEYPREFIX
hoodie.datasource.write.drop.partition.columns
When set to true, will not write the partition columns into hudi. By default, false.
Default Value: false (Optional)
Config Param: DROP_PARTITION_COLUMNS
hoodie.datasource.hive_sync.enable
When set to true, register/sync the table to Apache Hive metastore.
Default Value: false (Optional)
Config Param: HIVE_SYNC_ENABLED
hoodie.datasource.hive_sync.table
The name of the destination table that we should sync the hudi table to.
Default Value: unknown (Optional)
Config Param: HIVE_TABLE
hoodie.datasource.hive_sync.ignore_exceptions
Ignore exceptions when syncing with Hive.
Default Value: false (Optional)
Config Param: HIVE_IGNORE_EXCEPTIONS
hoodie.datasource.hive_sync.use_pre_apache_input_format
Flag to choose InputFormat under com.uber.hoodie package instead of org.apache.hudi package. Use this when you are in the process of migrating from com.uber.hoodie to org.apache.hudi. Stop using this after you migrated the table definition to org.apache.hudi input format
Default Value: false (Optional)
Config Param: HIVE_USE_PRE_APACHE_INPUT_FORMAT
hoodie.datasource.write.table.type
The table type for the underlying data, for this write. This can’t change between writes.
Default Value: COPY_ON_WRITE (Optional)
Config Param: TABLE_TYPE
hoodie.datasource.write.row.writer.enable
When set to true, will perform write operations directly using the spark native
Row
representation, avoiding any additional conversion costs.
Default Value: true (Optional)
Config Param: ENABLE_ROW_WRITER
hoodie.datasource.write.hive_style_partitioning
Flag to indicate whether to use Hive style partitioning. If set true, the names of partition folders follow <partition_column_name>=<partition_value> format. By default false (the names of partition folders are only partition values).
Default Value: false (Optional)
Config Param: HIVE_STYLE_PARTITIONING
hoodie.datasource.meta_sync.condition.sync
If true, only sync on conditions like schema change or partition change.
Default Value: false (Optional)
Config Param: HIVE_CONDITIONAL_SYNC
hoodie.datasource.hive_sync.mode
Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.
Default Value: N/A (Required)
Config Param: HIVE_SYNC_MODE
hoodie.datasource.write.table.name
Table name for the datasource write. Also used to register the table into meta stores.
Default Value: N/A (Required)
Config Param: TABLE_NAME
hoodie.datasource.hive_sync.base_file_format
Base file format for the sync.
Default Value: PARQUET (Optional)
Config Param: HIVE_BASE_FILE_FORMAT
hoodie.deltastreamer.source.kafka.value.deserializer.class
This class is used by kafka client to deserialize the records
Default Value: io.confluent.kafka.serializers.KafkaAvroDeserializer (Optional)
Config Param: KAFKA_AVRO_VALUE_DESERIALIZER_CLASS
Since Version: 0.9.0
hoodie.datasource.hive_sync.metastore.uris
Hive metastore url
Default Value: thrift://localhost:9083 (Optional)
Config Param: METASTORE_URIS
hoodie.datasource.write.insert.drop.duplicates
If set to true, filters out all duplicate records from incoming dataframe, during insert operations.
Default Value: false (Optional)
Config Param: INSERT_DROP_DUPS
hoodie.datasource.write.partitions.to.delete
Comma separated list of partitions to delete
Default Value: N/A (Required)
Config Param: PARTITIONS_TO_DELETE
PreCommit Validator Configurations
The following set of configurations help validate new data before commits.
Config Class
: org.apache.hudi.config.HoodiePreCommitValidatorConfig
hoodie.precommit.validators.single.value.sql.queries
Spark SQL queries to run on table before committing new data to validate state after commit.Multiple queries separated by ';' delimiter are supported.Expected result is included as part of query separated by '#'. Example query: 'query1#result1:query2#result2'Note <TABLE_NAME> variable is expected to be present in query.
Default Value: (Optional)
Config Param: SINGLE_VALUE_SQL_QUERIES
hoodie.precommit.validators.equality.sql.queries
Spark SQL queries to run on table before committing new data to validate state before and after commit. Multiple queries separated by ';' delimiter are supported. Example: "select count(*) from <TABLE_NAME> Note <TABLE_NAME> is replaced by table state before and after commit.
Default Value: (Optional)
Config Param: EQUALITY_SQL_QUERIES
hoodie.precommit.validators
Comma separated list of class names that can be invoked to validate commit
Default Value: (Optional)
Config Param: VALIDATOR_CLASS_NAMES
hoodie.precommit.validators.inequality.sql.queries
Spark SQL queries to run on table before committing new data to validate state before and after commit.Multiple queries separated by ';' delimiter are supported.Example query: 'select count(*) from <TABLE_NAME> where col=null'Note <TABLE_NAME> variable is expected to be present in query.
Default Value: (Optional)
Config Param: INEQUALITY_SQL_QUERIES
Flink Sql Configs
These configs control the Hudi Flink SQL source/sink connectors, providing ability to define record keys, pick out the write operation, specify how to merge records, enable/disable asynchronous compaction or choosing query type to read.
Flink Options
Flink jobs using the SQL can be configured through the options in WITH clause. The actual datasource level configs are listed below.
Config Class
: org.apache.hudi.configuration.FlinkOptions
compaction.trigger.strategy
Strategy to trigger compaction, options are 'num_commits': trigger compaction when reach N delta commits; 'time_elapsed': trigger compaction when time elapsed > N seconds since last compaction; 'num_and_time': trigger compaction when both NUM_COMMITS and TIME_ELAPSED are satisfied; 'num_or_time': trigger compaction when NUM_COMMITS or TIME_ELAPSED is satisfied. Default is 'num_commits'
Default Value: num_commits (Optional)
Config Param: COMPACTION_TRIGGER_STRATEGY
index.state.ttl
Index state ttl in days, default stores the index permanently
Default Value: 0.0 (Optional)
Config Param: INDEX_STATE_TTL
hive_sync.serde_properties
Serde properties to hive table, the data format is k1=v1 k2=v2
Default Value: N/A (Required)
Config Param: HIVE_SYNC_TABLE_SERDE_PROPERTIES
hive_sync.table
Table name for hive sync, default 'unknown'
Default Value: unknown (Optional)
Config Param: HIVE_SYNC_TABLE
write.payload.class
Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. This will render any value set for the option in-effective
Default Value: org.apache.hudi.common.model.OverwriteWithLatestAvroPayload (Optional)
Config Param: PAYLOAD_CLASS_NAME
compaction.tasks
Parallelism of tasks that do actual compaction, default is 4
Default Value: 4 (Optional)
Config Param: COMPACTION_TASKS
hoodie.datasource.write.hive_style_partitioning
Whether to use Hive style partitioning. If set true, the names of partition folders follow <partition_column_name>=<partition_value> format. By default false (the names of partition folders are only partition values)
Default Value: false (Optional)
Config Param: HIVE_STYLE_PARTITIONING
table.type
Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READ
Default Value: COPY_ON_WRITE (Optional)
Config Param: TABLE_TYPE
hive_sync.auto_create_db
Auto create hive database if it does not exists, default true
Default Value: true (Optional)
Config Param: HIVE_SYNC_AUTO_CREATE_DB
compaction.timeout.seconds
Max timeout time in seconds for online compaction to rollback, default 20 minutes
Default Value: 1200 (Optional)
Config Param: COMPACTION_TIMEOUT_SECONDS
hive_sync.username
Username for hive sync, default 'hive'
Default Value: hive (Optional)
Config Param: HIVE_SYNC_USERNAME
write.sort.memory
Sort memory in MB, default 128MB
Default Value: 128 (Optional)
Config Param: WRITE_SORT_MEMORY
write.bulk_insert.shuffle_input
Whether to shuffle the inputs by specific fields for bulk insert tasks, default true
Default Value: true (Optional)
Config Param: WRITE_BULK_INSERT_SHUFFLE_INPUT
write.retry.times
Flag to indicate how many times streaming job should retry for a failed checkpoint batch. By default 3
Default Value: 3 (Optional)
Config Param: RETRY_TIMES
metadata.enabled
Enable the internal metadata table which serves table metadata like level file listings, default disabled
Default Value: false (Optional)
Config Param: METADATA_ENABLED
write.parquet.max.file.size
Target size for parquet files produced by Hudi write phases. For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.
Default Value: 120 (Optional)
Config Param: WRITE_PARQUET_MAX_FILE_SIZE
clustering.plan.strategy.daybased.skipfromlatest.partitions
Number of partitions to skip from latest when choosing partitions to create ClusteringPlan
Default Value: 0 (Optional)
Config Param: CLUSTERING_PLAN_STRATEGY_SKIP_PARTITIONS_FROM_LATEST
hoodie.bucket.index.hash.field
Index key field. Value to be used as hashing to find the bucket ID. Should be a subset of or equal to the recordKey fields. Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using the dot notation eg:
a.b.c
Default Value: (Optional)
Config Param: INDEX_KEY_FIELD
hoodie.bucket.index.num.buckets
Hudi bucket number per partition. Only affected if using Hudi bucket index.
Default Value: 4 (Optional)
Config Param: BUCKET_INDEX_NUM_BUCKETS
hive_sync.mode
Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql, default 'jdbc'
Default Value: jdbc (Optional)
Config Param: HIVE_SYNC_MODE
write.retry.interval.ms
Flag to indicate how long (by millisecond) before a retry should issued for failed checkpoint batch. By default 2000 and it will be doubled by every retry
Default Value: 2000 (Optional)
Config Param: RETRY_INTERVAL_MS
write.partition.format
Partition path format, only valid when 'write.datetime.partitioning' is true, default is:
- 'yyyyMMddHH' for timestamp(3) WITHOUT TIME ZONE, LONG, FLOAT, DOUBLE, DECIMAL;
- 'yyyyMMdd' for DATE and INT.
Default Value: N/A (Required)
Config Param: PARTITION_FORMAT
clustering.async.enabled
Async Clustering, default false
Default Value: false (Optional)
Config Param: CLUSTERING_ASYNC_ENABLED
clustering.plan.partition.filter.mode
Partition filter mode used in the creation of clustering plan. Available values are - NONE: do not filter table partition and thus the clustering plan will include all partitions that have clustering candidate.RECENT_DAYS: keep a continuous range of partitions, worked together with configs 'hoodie.clustering.plan.strategy.daybased.lookback.partitions' and 'hoodie.clustering.plan.strategy.daybased.skipfromlatest.partitions.SELECTED_PARTITIONS: keep partitions that are in the specified range ['hoodie.clustering.plan.strategy.cluster.begin.partition', 'hoodie.clustering.plan.strategy.cluster.end.partition'].
Default Value: NONE (Optional)
Config Param: CLUSTERING_PLAN_PARTITION_FILTER_MODE_NAME
hive_sync.db
Database name for hive sync, default 'default'
Default Value: default (Optional)
Config Param: HIVE_SYNC_DB
clustering.plan.strategy.sort.columns
Columns to sort the data by when clustering
Default Value: (Optional)
Config Param: CLUSTERING_SORT_COLUMNS
compaction.schedule.enabled
Schedule the compaction plan, enabled by default for MOR
Default Value: true (Optional)
Config Param: COMPACTION_SCHEDULE_ENABLED
hive_sync.partition_extractor_class
Tool to extract the partition value from HDFS path, default 'SlashEncodedDayPartitionValueExtractor'
Default Value: org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor (Optional)
Config Param: HIVE_SYNC_PARTITION_EXTRACTOR_CLASS_NAME
write.precombine
Flag to indicate whether to drop duplicates before insert/upsert. By default these cases will accept duplicates, to gain extra performance:
- insert operation;
- upsert for MOR table, the MOR table deduplicate on reading
Default Value: false (Optional)
Config Param: PRE_COMBINE
write.batch.size
Batch buffer size in MB to flush data into the underneath filesystem, default 256MB
Default Value: 256.0 (Optional)
Config Param: WRITE_BATCH_SIZE
hoodie.datasource.write.keygenerator.class
Key generator class, that implements will extract the key out of incoming record
Default Value: N/A (Required)
Config Param: KEYGEN_CLASS_NAME
index.global.enabled
Whether to update index for the old partition path if same key record with different partition path came in, default true
Default Value: true (Optional)
Config Param: INDEX_GLOBAL_ENABLED
clustering.delta_commits
Max delta commits needed to trigger clustering, default 4 commits
Default Value: 4 (Optional)
Config Param: CLUSTERING_DELTA_COMMITS
path
Base path for the target hoodie table. The path would be created if it does not exist, otherwise a Hoodie table expects to be initialized successfully
Default Value: N/A (Required)
Config Param: PATH
index.bootstrap.enabled
Whether to bootstrap the index state from existing hoodie table, default false
Default Value: false (Optional)
Config Param: INDEX_BOOTSTRAP_ENABLED
read.streaming.skip_compaction
Whether to skip compaction instants for streaming read, there are two cases that this option can be used to avoid reading duplicates:
- you are definitely sure that the consumer reads faster than any compaction instants, usually with delta time compaction strategy that is long enough, for e.g, one week;
- changelog mode is enabled, this option is a solution to keep data integrity
Default Value: false (Optional)
Config Param: READ_STREAMING_SKIP_COMPACT
compaction.async.enabled
Async Compaction, enabled by default for MOR
Default Value: true (Optional)
Config Param: COMPACTION_ASYNC_ENABLED
hive_sync.ignore_exceptions
Ignore exceptions during hive synchronization, default false
Default Value: false (Optional)
Config Param: HIVE_SYNC_IGNORE_EXCEPTIONS
hive_sync.table_properties
Additional properties to store with table, the data format is k1=v1 k2=v2
Default Value: N/A (Required)
Config Param: HIVE_SYNC_TABLE_PROPERTIES
write.ignore.failed
Flag to indicate whether to ignore any non exception error (e.g. writestatus error). within a checkpoint batch. By default true (in favor of streaming progressing over data integrity)
Default Value: true (Optional)
Config Param: IGNORE_FAILED
write.commit.ack.timeout
Timeout limit for a writer task after it finishes a checkpoint and waits for the instant commit success, only for internal use
Default Value: -1 (Optional)
Config Param: WRITE_COMMIT_ACK_TIMEOUT
write.operation
The write operation, that this write should do
Default Value: upsert (Optional)
Config Param: OPERATION
hoodie.datasource.write.partitionpath.field
Partition path field. Value to be used at the
partitionPath
component ofHoodieKey
. Actual value obtained by invoking .toString(), default ''
Default Value: (Optional)
Config Param: PARTITION_PATH_FIELD
write.bucket_assign.tasks
Parallelism of tasks that do bucket assign, default is the parallelism of the execution environment
Default Value: N/A (Required)
Config Param: BUCKET_ASSIGN_TASKS
compaction.delta_commits
Max delta commits needed to trigger compaction, default 5 commits
Default Value: 5 (Optional)
Config Param: COMPACTION_DELTA_COMMITS
partition.default_name
The default partition name in case the dynamic partition column value is null/empty string
Default Value: HIVE_DEFAULT_PARTITION (Optional)
Config Param: PARTITION_DEFAULT_NAME
write.bulk_insert.sort_input
Whether to sort the inputs by specific fields for bulk insert tasks, default true
Default Value: true (Optional)
Config Param: WRITE_BULK_INSERT_SORT_INPUT
clustering.plan.strategy.small.file.limit
Files smaller than the size specified here are candidates for clustering, default 600 MB
Default Value: 600 (Optional)
Config Param: CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT
clustering.schedule.enabled
Schedule the cluster plan, default false
Default Value: false (Optional)
Config Param: CLUSTERING_SCHEDULE_ENABLED
compaction.target_io
Target IO in MB for per compaction (both read and write), default 500 GB
Default Value: 512000 (Optional)
Config Param: COMPACTION_TARGET_IO
clustering.plan.strategy.class
Config to provide a strategy class (subclass of ClusteringPlanStrategy) to create clustering plan i.e select what file groups are being clustered. Default strategy, looks at the last N (determined by clustering.plan.strategy.daybased.lookback.partitions) day based partitions picks the small file slices within those partitions.
Default Value: org.apache.hudi.client.clustering.plan.strategy.FlinkSizeBasedClusteringPlanStrategy (Optional)
Config Param: CLUSTERING_PLAN_STRATEGY_CLASS
write.log_block.size
Max log block size in MB for log file, default 128MB
Default Value: 128 (Optional)
Config Param: WRITE_LOG_BLOCK_SIZE
write.tasks
Parallelism of tasks that do actual write, default is 4
Default Value: 4 (Optional)
Config Param: WRITE_TASKS
clean.async.enabled
Whether to cleanup the old commits immediately on new commits, enabled by default
Default Value: true (Optional)
Config Param: CLEAN_ASYNC_ENABLED
clean.retain_commits
Number of commits to retain. So data will be retained for num_of_commits * time_between_commits (scheduled). This also directly translates into how much you can incrementally pull on this table, default 30
Default Value: 30 (Optional)
Config Param: CLEAN_RETAIN_COMMITS
archive.max_commits
Max number of commits to keep before archiving older commits into a sequential log, default 50
Default Value: 50 (Optional)
Config Param: ARCHIVE_MAX_COMMITS
write.index_bootstrap.tasks
Parallelism of tasks that do index bootstrap, default is the parallelism of the execution environment
Default Value: N/A (Required)
Config Param: INDEX_BOOTSTRAP_TASKS
write.task.max.size
Maximum memory in MB for a write task, when the threshold hits, it flushes the max size data bucket to avoid OOM, default 1GB
Default Value: 1024.0 (Optional)
Config Param: WRITE_TASK_MAX_SIZE
hoodie.datasource.write.recordkey.field
Record key field. Value to be used as the
recordKey
component ofHoodieKey
. Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using the dot notation eg:a.b.c
Default Value: uuid (Optional)
Config Param: RECORD_KEY_FIELD
compaction.delta_seconds
Max delta seconds time needed to trigger compaction, default 1 hour
Default Value: 3600 (Optional)
Config Param: COMPACTION_DELTA_SECONDS
hive_sync.partition_fields
Partition fields for hive sync, default ''
Default Value: (Optional)
Config Param: HIVE_SYNC_PARTITION_FIELDS
read.streaming.enabled
Whether to read as streaming source, default false
Default Value: false (Optional)
Config Param: READ_AS_STREAMING
hoodie.datasource.write.keygenerator.type
Key generator type, that implements will extract the key out of incoming record. Note This is being actively worked on. Please use
hoodie.datasource.write.keygenerator.class
instead.
Default Value: SIMPLE (Optional)
Config Param: KEYGEN_TYPE
clean.retain_file_versions
Number of file versions to retain. default 5
Default Value: 5 (Optional)
Config Param: CLEAN_RETAIN_FILE_VERSIONS
compaction.max_memory
Max memory in MB for compaction spillable map, default 100MB
Default Value: 100 (Optional)
Config Param: COMPACTION_MAX_MEMORY
hive_sync.support_timestamp
INT64 with original type TIMESTAMP_MICROS is converted to hive timestamp type. Disabled by default for backward compatibility.
Default Value: true (Optional)
Config Param: HIVE_SYNC_SUPPORT_TIMESTAMP
hive_sync.skip_ro_suffix
Skip the _ro suffix for Read optimized table when registering, default false
Default Value: false (Optional)
Config Param: HIVE_SYNC_SKIP_RO_SUFFIX
metadata.compaction.delta_commits
Max delta commits for metadata table to trigger compaction, default 10
Default Value: 10 (Optional)
Config Param: METADATA_COMPACTION_DELTA_COMMITS
hive_sync.assume_date_partitioning
Assume partitioning is yyyy/mm/dd, default false
Default Value: false (Optional)
Config Param: HIVE_SYNC_ASSUME_DATE_PARTITION
write.parquet.block.size
Parquet RowGroup size. It's recommended to make this large enough that scan costs can be amortized by packing enough column values into a single row group.
Default Value: 120 (Optional)
Config Param: WRITE_PARQUET_BLOCK_SIZE
clustering.plan.strategy.target.file.max.bytes
Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB
Default Value: 1073741824 (Optional)
Config Param: CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES
clustering.tasks
Parallelism of tasks that do actual clustering, default is 4
Default Value: 4 (Optional)
Config Param: CLUSTERING_TASKS
hive_sync.enable
Asynchronously sync Hive meta to HMS, default false
Default Value: false (Optional)
Config Param: HIVE_SYNC_ENABLED
changelog.enabled
Whether to keep all the intermediate changes, we try to keep all the changes of a record when enabled: 1). The sink accept the UPDATE_BEFORE message; 2). The source try to emit every changes of a record. The semantics is best effort because the compaction job would finally merge all changes of a record into one. default false to have UPSERT semantics
Default Value: false (Optional)
Config Param: CHANGELOG_ENABLED
read.streaming.check-interval
Check interval for streaming read of SECOND, default 1 minute
Default Value: 60 (Optional)
Config Param: READ_STREAMING_CHECK_INTERVAL
hoodie.datasource.merge.type
For Snapshot query on merge on read table. Use this key to define how the payloads are merged, in 1) skip_merge: read the base file records plus the log file records; 2) payload_combine: read the base file records first, for each record in base file, checks whether the key is in the log file records(combines the two records with same key for base and log file records), then read the left log file records
Default Value: payload_combine (Optional)
Config Param: MERGE_TYPE
read.tasks
Parallelism of tasks that do actual read, default is 4
Default Value: 4 (Optional)
Config Param: READ_TASKS
read.end-commit
End commit instant for reading, the commit time format should be 'yyyyMMddHHmmss'
Default Value: N/A (Required)
Config Param: READ_END_COMMIT
write.log.max.size
Maximum size allowed in MB for a log file before it is rolled over to the next version, default 1GB
Default Value: 1024 (Optional)
Config Param: WRITE_LOG_MAX_SIZE
clustering.plan.strategy.daybased.lookback.partitions
Number of partitions to list to create ClusteringPlan, default is 2
Default Value: 2 (Optional)
Config Param: CLUSTERING_TARGET_PARTITIONS
hive_sync.file_format
File format for hive sync, default 'PARQUET'
Default Value: PARQUET (Optional)
Config Param: HIVE_SYNC_FILE_FORMAT
clustering.plan.strategy.max.num.groups
Maximum number of groups to create as part of ClusteringPlan. Increasing groups will increase parallelism, default is 30
Default Value: 30 (Optional)
Config Param: CLUSTERING_MAX_NUM_GROUPS
index.type
Index type of Flink write job, default is using state backed index.
Default Value: FLINK_STATE (Optional)
Config Param: INDEX_TYPE
read.data.skipping.enabled
Enables data-skipping allowing queries to leverage indexes to reduce the search space byskipping over files
Default Value: false (Optional)
Config Param: READ_DATA_SKIPPING_ENABLED
clean.policy
Clean policy to manage the Hudi table. Available option: KEEP_LATEST_COMMITS, KEEP_LATEST_FILE_VERSIONS, KEEP_LATEST_BY_HOURS.Default is KEEP_LATEST_COMMITS.
Default Value: KEEP_LATEST_COMMITS (Optional)
Config Param: CLEAN_POLICY
hive_sync.password
Password for hive sync, default 'hive'
Default Value: hive (Optional)
Config Param: HIVE_SYNC_PASSWORD
hive_sync.use_jdbc
Use JDBC when hive synchronization is enabled, default true
Default Value: true (Optional)
Config Param: HIVE_SYNC_USE_JDBC
hive_sync.jdbc_url
Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'
Default Value: jdbc:hive2://localhost:10000 (Optional)
Config Param: HIVE_SYNC_JDBC_URL
read.start-commit
Start commit instant for reading, the commit time format should be 'yyyyMMddHHmmss', by default reading from the latest instant for streaming read
Default Value: N/A (Required)
Config Param: READ_START_COMMIT
archive.min_commits
Min number of commits to keep before archiving older commits into a sequential log, default 40
Default Value: 40 (Optional)
Config Param: ARCHIVE_MIN_COMMITS
index.partition.regex
Whether to load partitions in state if partition path matching, default
*
Default Value: .* (Optional)
Config Param: INDEX_PARTITION_REGEX
hoodie.table.name
Table name to register to Hive metastore
Default Value: N/A (Required)
Config Param: TABLE_NAME
hoodie.datasource.write.partitionpath.urlencode
Whether to encode the partition path url, default false
Default Value: false (Optional)
Config Param: URL_ENCODE_PARTITIONING
source.avro-schema.path
Source avro schema file path, the parsed schema is used for deserialization
Default Value: N/A (Required)
Config Param: SOURCE_AVRO_SCHEMA_PATH
write.insert.cluster
Whether to merge small files for insert mode, if true, the write throughput will decrease because the read/write of existing small file, only valid for COW table, default false
Default Value: false (Optional)
Config Param: INSERT_CLUSTER
source.avro-schema
Source avro schema string, the parsed schema is used for deserialization
Default Value: N/A (Required)
Config Param: SOURCE_AVRO_SCHEMA
hive_sync.conf.dir
The hive configuration directory, where the hive-site.xml lies in, the file should be put on the client machine
Default Value: N/A (Required)
Config Param: HIVE_SYNC_CONF_DIR
write.rate.limit
Write record rate limit per second to prevent traffic jitter and improve stability, default 0 (no limit)
Default Value: 0 (Optional)
Config Param: WRITE_RATE_LIMIT
clean.retain_hours
Number of hours for which commits need to be retained. This config provides a more flexible option ascompared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group, corresponding to commits with commit times older than the configured number of hours to be retained are cleaned.
Default Value: 24 (Optional)
Config Param: CLEAN_RETAIN_HOURS
read.utc-timezone
Use UTC timezone or local timezone to the conversion between epoch time and LocalDateTime. Hive 0.x/1.x/2.x use local timezone. But Hive 3.x use UTC timezone, by default true
Default Value: true (Optional)
Config Param: UTC_TIMEZONE
hoodie.datasource.query.type
Decides how data files need to be read, in 1) Snapshot mode (obtain latest view, based on row & columnar data); 2) incremental mode (new data since an instantTime); 3) Read Optimized mode (obtain latest view, based on columnar data) .Default: snapshot
Default Value: snapshot (Optional)
Config Param: QUERY_TYPE
write.precombine.field
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(..)
Default Value: ts (Optional)
Config Param: PRECOMBINE_FIELD
write.parquet.page.size
Parquet page size. Page is the unit of read within a parquet file. Within a block, pages are compressed separately.
Default Value: 1 (Optional)
Config Param: WRITE_PARQUET_PAGE_SIZE
hive_sync.metastore.uris
Metastore uris for hive sync, default ''
Default Value: (Optional)
Config Param: HIVE_SYNC_METASTORE_URIS
write.merge.max_memory
Max memory in MB for merge, default 100MB
Default Value: 100 (Optional)
Config Param: WRITE_MERGE_MAX_MEMORY
Write Client Configs
Internally, the Hudi datasource uses a RDD based HoodieWriteClient API to actually perform writes to storage. These configs provide deep control over lower level aspects like file sizing, compression, parallelism, compaction, write schema, cleaning etc. Although Hudi provides sane defaults, from time-time these configs may need to be tweaked to optimize for specific workloads.
Layout Configs
Configurations that control storage layout and data distribution, which defines how the files are organized within a table.
Config Class
: org.apache.hudi.config.HoodieLayoutConfig
hoodie.storage.layout.type
Type of storage layout. Possible options are [DEFAULT | BUCKET]
Default Value: DEFAULT (Optional)
Config Param: LAYOUT_TYPE
hoodie.storage.layout.partitioner.class
Partitioner class, it is used to distribute data in a specific way.
Default Value: N/A (Required)
Config Param: LAYOUT_PARTITIONER_CLASS_NAME
Write commit callback configs
Controls callback behavior into HTTP endpoints, to push notifications on commits on hudi tables.
Config Class
: org.apache.hudi.config.HoodieWriteCommitCallbackConfig
hoodie.write.commit.callback.on
Turn commit callback on/off. off by default.
Default Value: false (Optional)
Config Param: TURN_CALLBACK_ON
Since Version: 0.6.0
hoodie.write.commit.callback.http.url
Callback host to be sent along with callback messages
Default Value: N/A (Required)
Config Param: CALLBACK_HTTP_URL
Since Version: 0.6.0
hoodie.write.commit.callback.http.timeout.seconds
Callback timeout in seconds. 3 by default
Default Value: 3 (Optional)
Config Param: CALLBACK_HTTP_TIMEOUT_IN_SECONDS
Since Version: 0.6.0
hoodie.write.commit.callback.class
Full path of callback class and must be a subclass of HoodieWriteCommitCallback class, org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback by default
Default Value: org.apache.hudi.callback.impl.HoodieWriteCommitHttpCallback (Optional)
Config Param: CALLBACK_CLASS_NAME
Since Version: 0.6.0
hoodie.write.commit.callback.http.api.key
Http callback API key. hudi_write_commit_http_callback by default
Default Value: hudi_write_commit_http_callback (Optional)
Config Param: CALLBACK_HTTP_API_KEY_VALUE
Since Version: 0.6.0
Clean Configs
Cleaning (reclamation of older/unused file groups/slices).
Config Class
: org.apache.hudi.config.HoodieCleanConfig
hoodie.cleaner.fileversions.retained
When KEEP_LATEST_FILE_VERSIONS cleaning policy is used, the minimum number of file slices to retain in each file group, during cleaning.
Default Value: 3 (Optional)
Config Param: CLEANER_FILE_VERSIONS_RETAINED
hoodie.clean.max.commits
Number of commits after the last clean operation, before scheduling of a new clean is attempted.
Default Value: 1 (Optional)
Config Param: CLEAN_MAX_COMMITS
hoodie.clean.allow.multiple
Allows scheduling/executing multiple cleans by enabling this config. If users prefer to strictly ensure clean requests should be mutually exclusive, .i.e. a 2nd clean will not be scheduled if another clean is not yet completed to avoid repeat cleaning of same files, they might want to disable this config.
Default Value: true (Optional)
Config Param: ALLOW_MULTIPLE_CLEANS
Since Version: 0.11.0
hoodie.clean.automatic
When enabled, the cleaner table service is invoked immediately after each commit, to delete older file slices. It's recommended to enable this, to ensure metadata and data storage growth is bounded.
Default Value: true (Optional)
Config Param: AUTO_CLEAN
hoodie.cleaner.parallelism
Parallelism for the cleaning operation. Increase this if cleaning becomes slow.
Default Value: 200 (Optional)
Config Param: CLEANER_PARALLELISM_VALUE
hoodie.cleaner.incremental.mode
When enabled, the plans for each cleaner service run is computed incrementally off the events in the timeline, since the last cleaner run. This is much more efficient than obtaining listings for the full table for each planning (even with a metadata table).
Default Value: true (Optional)
Config Param: CLEANER_INCREMENTAL_MODE_ENABLE
hoodie.clean.async
Only applies when hoodie.clean.automatic is turned on. When turned on runs cleaner async with writing, which can speed up overall write performance.
Default Value: false (Optional)
Config Param: ASYNC_CLEAN
hoodie.clean.trigger.strategy
Controls how cleaning is scheduled. Valid options: NUM_COMMITS
Default Value: NUM_COMMITS (Optional)
Config Param: CLEAN_TRIGGER_STRATEGY
hoodie.cleaner.delete.bootstrap.base.file
When set to true, cleaner also deletes the bootstrap base file when it's skeleton base file is cleaned. Turn this to true, if you want to ensure the bootstrap dataset storage is reclaimed over time, as the table receives updates/deletes. Another reason to turn this on, would be to ensure data residing in bootstrap base files are also physically deleted, to comply with data privacy enforcement processes.
Default Value: false (Optional)
Config Param: CLEANER_BOOTSTRAP_BASE_FILE_ENABLE
hoodie.cleaner.hours.retained
Number of hours for which commits need to be retained. This config provides a more flexible option ascompared to number of commits retained for cleaning service. Setting this property ensures all the files, but the latest in a file group, corresponding to commits with commit times older than the configured number of hours to be retained are cleaned.
Default Value: 24 (Optional)
Config Param: CLEANER_HOURS_RETAINED
hoodie.cleaner.commits.retained
Number of commits to retain, without cleaning. This will be retained for num_of_commits * time_between_commits (scheduled). This also directly translates into how much data retention the table supports for incremental queries.
Default Value: 10 (Optional)
Config Param: CLEANER_COMMITS_RETAINED
hoodie.cleaner.policy.failed.writes
Cleaning policy for failed writes to be used. Hudi will delete any files written by failed writes to re-claim space. Choose to perform this rollback of failed writes eagerly before every writer starts (only supported for single writer) or lazily by the cleaner (required for multi-writers)
Default Value: EAGER (Optional)
Config Param: FAILED_WRITES_CLEANER_POLICY
hoodie.cleaner.policy
Cleaning policy to be used. The cleaner service deletes older file slices files to re-claim space. By default, cleaner spares the file slices written by the last N commits, determined by hoodie.cleaner.commits.retained Long running query plans may often refer to older file slices and will break if those are cleaned, before the query has had a chance to run. So, it is good to make sure that the data is retained for more than the maximum query execution time
Default Value: KEEP_LATEST_COMMITS (Optional)
Config Param: CLEANER_POLICY
Metastore Configs
Configurations used by the Hudi Metastore.
Config Class
: org.apache.hudi.common.config.HoodieMetastoreConfig
hoodie.metastore.uris
Metastore server uris
Default Value: thrift://localhost:9090 (Optional)
Config Param: METASTORE_URLS
hoodie.metastore.enable
Use metastore server to store hoodie table metadata
Default Value: false (Optional)
Config Param: METASTORE_ENABLE
hoodie.metastore.connect.retries
Number of retries while opening a connection to metastore
Default Value: 3 (Optional)
Config Param: METASTORE_CONNECTION_RETRIES
hoodie.metastore.connect.retry.delay
Number of seconds for the client to wait between consecutive connection attempts
Default Value: 1 (Optional)
Config Param: METASTORE_CONNECTION_RETRY_DELAY
Table Configurations
Configurations that persist across writes and read on a Hudi table like base, log file formats, table name, creation schema, table version layouts. Configurations are loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path and rarely changes during the lifetime of the table. Writers/Queries' configurations are validated against these each time for compatibility.
Config Class
: org.apache.hudi.common.table.HoodieTableConfig
hoodie.table.precombine.field
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.
Default Value: N/A (Required)
Config Param: PRECOMBINE_FIELD
hoodie.archivelog.folder
path under the meta folder, to store archived timeline instants at.
Default Value: archived (Optional)
Config Param: ARCHIVELOG_FOLDER
hoodie.table.type
The table type for the underlying data, for this write. This can’t change between writes.
Default Value: COPY_ON_WRITE (Optional)
Config Param: TYPE
hoodie.table.timeline.timezone
User can set hoodie commit timeline timezone, such as utc, local and so on. local is default
Default Value: LOCAL (Optional)
Config Param: TIMELINE_TIMEZONE
hoodie.partition.metafile.use.base.format
If true, partition metafiles are saved in the same format as base-files for this dataset (e.g. Parquet / ORC). If false (default) partition metafiles are saved as properties files.
Default Value: false (Optional)
Config Param: PARTITION_METAFILE_USE_BASE_FORMAT
hoodie.table.checksum
Table checksum is used to guard against partial writes in HDFS. It is added as the last entry in hoodie.properties and then used to validate while reading table config.
Default Value: N/A (Required)
Config Param: TABLE_CHECKSUM
Since Version: 0.11.0
hoodie.table.create.schema
Schema used when creating the table, for the first time.
Default Value: N/A (Required)
Config Param: CREATE_SCHEMA
hoodie.table.recordkey.fields
Columns used to uniquely identify the table. Concatenated values of these fields are used as the record key component of HoodieKey.
Default Value: N/A (Required)
Config Param: RECORDKEY_FIELDS
hoodie.table.log.file.format
Log format used for the delta logs.
Default Value: HOODIE_LOG (Optional)
Config Param: LOG_FILE_FORMAT
hoodie.bootstrap.index.enable
Whether or not, this is a bootstrapped table, with bootstrap base data and an mapping index defined, default true.
Default Value: true (Optional)
Config Param: BOOTSTRAP_INDEX_ENABLE
hoodie.table.metadata.partitions
Comma-separated list of metadata partitions that have been completely built and in-sync with data table. These partitions are ready for use by the readers
Default Value: N/A (Required)
Config Param: TABLE_METADATA_PARTITIONS
Since Version: 0.11.0
hoodie.table.metadata.partitions.inflight
Comma-separated list of metadata partitions whose building is in progress. These partitions are not yet ready for use by the readers.
Default Value: N/A (Required)
Config Param: TABLE_METADATA_PARTITIONS_INFLIGHT
Since Version: 0.11.0
hoodie.table.partition.fields
Fields used to partition the table. Concatenated values of these fields are used as the partition path, by invoking toString()
Default Value: N/A (Required)
Config Param: PARTITION_FIELDS
hoodie.populate.meta.fields
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
Default Value: true (Optional)
Config Param: POPULATE_META_FIELDS
hoodie.compaction.payload.class
Payload class to use for performing compactions, i.e merge delta logs with current base file and then produce a new base file.
Default Value: org.apache.hudi.common.model.OverwriteWithLatestAvroPayload (Optional)
Config Param: PAYLOAD_CLASS_NAME
hoodie.bootstrap.index.class
Implementation to use, for mapping base files to bootstrap base file, that contain actual data.
Default Value: org.apache.hudi.common.bootstrap.index.HFileBootstrapIndex (Optional)
Config Param: BOOTSTRAP_INDEX_CLASS_NAME
hoodie.datasource.write.partitionpath.urlencode
Should we url encode the partition path value, before creating the folder structure.
Default Value: false (Optional)
Config Param: URL_ENCODE_PARTITIONING
hoodie.datasource.write.hive_style_partitioning
Flag to indicate whether to use Hive style partitioning. If set true, the names of partition folders follow <partition_column_name>=<partition_value> format. By default false (the names of partition folders are only partition values)
Default Value: false (Optional)
Config Param: HIVE_STYLE_PARTITIONING_ENABLE
hoodie.table.keygenerator.class
Key Generator class property for the hoodie table
Default Value: N/A (Required)
Config Param: KEY_GENERATOR_CLASS_NAME
hoodie.table.version
Version of table, used for running upgrade/downgrade steps between releases with potentially breaking/backwards compatible changes.
Default Value: ZERO (Optional)
Config Param: VERSION
hoodie.table.base.file.format
Base file format to store all the base file data.
Default Value: PARQUET (Optional)
Config Param: BASE_FILE_FORMAT
hoodie.bootstrap.base.path
Base path of the dataset that needs to be bootstrapped as a Hudi table
Default Value: N/A (Required)
Config Param: BOOTSTRAP_BASE_PATH
hoodie.datasource.write.drop.partition.columns
When set to true, will not write the partition columns into hudi. By default, false.
Default Value: false (Optional)
Config Param: DROP_PARTITION_COLUMNS
hoodie.database.name
Database name that will be used for incremental query.If different databases have the same table name during incremental query, we can set it to limit the table name under a specific database
Default Value: N/A (Required)
Config Param: DATABASE_NAME
hoodie.timeline.layout.version
Version of timeline used, by the table.
Default Value: N/A (Required)
Config Param: TIMELINE_LAYOUT_VERSION
hoodie.table.name
Table name that will be used for registering with Hive. Needs to be same across runs.
Default Value: N/A (Required)
Config Param: NAME
Memory Configurations
Controls memory usage for compaction and merges, performed internally by Hudi.
Config Class
: org.apache.hudi.config.HoodieMemoryConfig
hoodie.memory.merge.fraction
This fraction is multiplied with the user memory fraction (1 - spark.memory.fraction) to get a final fraction of heap space to use during merge
Default Value: 0.6 (Optional)
Config Param: MAX_MEMORY_FRACTION_FOR_MERGE
hoodie.memory.dfs.buffer.max.size
Property to control the max memory in bytes for dfs input stream buffer size
Default Value: 16777216 (Optional)
Config Param: MAX_DFS_STREAM_BUFFER_SIZE
hoodie.memory.writestatus.failure.fraction
Property to control how what fraction of the failed record, exceptions we report back to driver. Default is 10%. If set to 100%, with lot of failures, this can cause memory pressure, cause OOMs and mask actual data errors.
Default Value: 0.1 (Optional)
Config Param: WRITESTATUS_FAILURE_FRACTION
hoodie.memory.compaction.fraction
HoodieCompactedLogScanner reads logblocks, converts records to HoodieRecords and then merges these log blocks and records. At any point, the number of entries in a log block can be less than or equal to the number of entries in the corresponding parquet file. This can lead to OOM in the Scanner. Hence, a spillable map helps alleviate the memory pressure. Use this config to set the max allowable inMemory footprint of the spillable map
Default Value: 0.6 (Optional)
Config Param: MAX_MEMORY_FRACTION_FOR_COMPACTION
hoodie.memory.merge.max.size
Maximum amount of memory used in bytes for merge operations, before spilling to local storage.
Default Value: 1073741824 (Optional)
Config Param: MAX_MEMORY_FOR_MERGE
hoodie.memory.spillable.map.path
Default file path prefix for spillable map
Default Value: /tmp/ (Optional)
Config Param: SPILLABLE_MAP_BASE_PATH
hoodie.memory.compaction.max.size
Maximum amount of memory used in bytes for compaction operations in bytes , before spilling to local storage.
Default Value: N/A (Required)
Config Param: MAX_MEMORY_FOR_COMPACTION
DynamoDB based Locks Configurations
Configs that control DynamoDB based locking mechanisms required for concurrency control between writers to a Hudi table. Concurrency between Hudi's own table services are auto managed internally.
Config Class
: org.apache.hudi.config.DynamoDbBasedLockConfig
hoodie.write.lock.dynamodb.billing_mode
For DynamoDB based lock provider, by default it is PAY_PER_REQUEST mode
Default Value: PAY_PER_REQUEST (Optional)
Config Param: DYNAMODB_LOCK_BILLING_MODE
Since Version: 0.10.0
hoodie.write.lock.dynamodb.table
For DynamoDB based lock provider, the name of the DynamoDB table acting as lock table
Default Value: N/A (Required)
Config Param: DYNAMODB_LOCK_TABLE_NAME
Since Version: 0.10.0
hoodie.write.lock.dynamodb.region
For DynamoDB based lock provider, the region used in endpoint for Amazon DynamoDB service. Would try to first get it from AWS_REGION environment variable. If not find, by default use us-east-1
Default Value: us-east-1 (Optional)
Config Param: DYNAMODB_LOCK_REGION
Since Version: 0.10.0
hoodie.write.lock.dynamodb.partition_key
For DynamoDB based lock provider, the partition key for the DynamoDB lock table. Each Hudi dataset should has it's unique key so concurrent writers could refer to the same partition key. By default we use the Hudi table name specified to be the partition key
Default Value: N/A (Required)
Config Param: DYNAMODB_LOCK_PARTITION_KEY
Since Version: 0.10.0
hoodie.write.lock.dynamodb.write_capacity
For DynamoDB based lock provider, write capacity units when using PROVISIONED billing mode
Default Value: 10 (Optional)
Config Param: DYNAMODB_LOCK_WRITE_CAPACITY
Since Version: 0.10.0
hoodie.write.lock.dynamodb.table_creation_timeout
For DynamoDB based lock provider, the maximum number of milliseconds to wait for creating DynamoDB table
Default Value: 600000 (Optional)
Config Param: DYNAMODB_LOCK_TABLE_CREATION_TIMEOUT
Since Version: 0.10.0
hoodie.write.lock.dynamodb.read_capacity
For DynamoDB based lock provider, read capacity units when using PROVISIONED billing mode
Default Value: 20 (Optional)
Config Param: DYNAMODB_LOCK_READ_CAPACITY
Since Version: 0.10.0
hoodie.write.lock.dynamodb.endpoint_url
For DynamoDB based lock provider, the url endpoint used for Amazon DynamoDB service. Useful for development with a local dynamodb instance.
Default Value: N/A (Required)
Config Param: DYNAMODB_ENDPOINT_URL
Since Version: 0.10.1
Storage Configs
Configurations that control aspects around writing, sizing, reading base and log files.
Config Class
: org.apache.hudi.config.HoodieStorageConfig
hoodie.logfile.data.block.max.size
LogFile Data block max size in bytes. This is the maximum size allowed for a single data block to be appended to a log file. This helps to make sure the data appended to the log file is broken up into sizable blocks to prevent from OOM errors. This size should be greater than the JVM memory.
Default Value: 268435456 (Optional)
Config Param: LOGFILE_DATA_BLOCK_MAX_SIZE
hoodie.parquet.outputtimestamptype
Sets spark.sql.parquet.outputTimestampType. Parquet timestamp type to use when Spark writes data to Parquet files.
Default Value: TIMESTAMP_MICROS (Optional)
Config Param: PARQUET_OUTPUT_TIMESTAMP_TYPE
hoodie.orc.stripe.size
Size of the memory buffer in bytes for writing
Default Value: 67108864 (Optional)
Config Param: ORC_STRIPE_SIZE
hoodie.orc.block.size
ORC block size, recommended to be aligned with the target file size.
Default Value: 125829120 (Optional)
Config Param: ORC_BLOCK_SIZE
hoodie.orc.compression.codec
Compression codec to use for ORC base files.
Default Value: ZLIB (Optional)
Config Param: ORC_COMPRESSION_CODEC_NAME
hoodie.parquet.max.file.size
Target size in bytes for parquet files produced by Hudi write phases. For DFS, this needs to be aligned with the underlying filesystem block size for optimal performance.
Default Value: 125829120 (Optional)
Config Param: PARQUET_MAX_FILE_SIZE
hoodie.hfile.max.file.size
Target file size in bytes for HFile base files.
Default Value: 125829120 (Optional)
Config Param: HFILE_MAX_FILE_SIZE
hoodie.parquet.writelegacyformat.enabled
Sets spark.sql.parquet.writeLegacyFormat. If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values will be written in Parquet's fixed-length byte array format which other systems such as Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For example, decimals will be written in int-based format.
Default Value: false (Optional)
Config Param: PARQUET_WRITE_LEGACY_FORMAT_ENABLED
hoodie.parquet.block.size
Parquet RowGroup size in bytes. It's recommended to make this large enough that scan costs can be amortized by packing enough column values into a single row group.
Default Value: 125829120 (Optional)
Config Param: PARQUET_BLOCK_SIZE
hoodie.logfile.max.size
LogFile max size in bytes. This is the maximum size allowed for a log file before it is rolled over to the next version.
Default Value: 1073741824 (Optional)
Config Param: LOGFILE_MAX_SIZE
hoodie.parquet.dictionary.enabled
Whether to use dictionary encoding
Default Value: true (Optional)
Config Param: PARQUET_DICTIONARY_ENABLED
hoodie.hfile.block.size
Lower values increase the size in bytes of metadata tracked within HFile, but can offer potentially faster lookup times.
Default Value: 1048576 (Optional)
Config Param: HFILE_BLOCK_SIZE
hoodie.parquet.field_id.write.enabled
Would only be effective with Spark 3.3+. Sets spark.sql.parquet.fieldId.write.enabled. If enabled, Spark will write out parquet native field ids that are stored inside StructField's metadata as parquet.field.id to parquet files.
Default Value: true (Optional)
Config Param: PARQUET_FIELD_ID_WRITE_ENABLED
Since Version: 0.12.0
hoodie.parquet.page.size
Parquet page size in bytes. Page is the unit of read within a parquet file. Within a block, pages are compressed separately.
Default Value: 1048576 (Optional)
Config Param: PARQUET_PAGE_SIZE
hoodie.hfile.compression.algorithm
Compression codec to use for hfile base files.
Default Value: GZ (Optional)
Config Param: HFILE_COMPRESSION_ALGORITHM_NAME
hoodie.orc.max.file.size
Target file size in bytes for ORC base files.
Default Value: 125829120 (Optional)
Config Param: ORC_FILE_MAX_SIZE
hoodie.logfile.data.block.format
Format of the data block within delta logs. Following formats are currently supported "avro", "hfile", "parquet"
Default Value: N/A (Required)
Config Param: LOGFILE_DATA_BLOCK_FORMAT
hoodie.logfile.to.parquet.compression.ratio
Expected additional compression as records move from log files to parquet. Used for merge_on_read table to send inserts into log files & control the size of compacted parquet file.
Default Value: 0.35 (Optional)
Config Param: LOGFILE_TO_PARQUET_COMPRESSION_RATIO_FRACTION
hoodie.parquet.compression.ratio
Expected compression of parquet data used by Hudi, when it tries to size new parquet files. Increase this value, if bulk_insert is producing smaller than expected sized files
Default Value: 0.1 (Optional)
Config Param: PARQUET_COMPRESSION_RATIO_FRACTION
hoodie.parquet.compression.codec
Compression Codec for parquet files
Default Value: gzip (Optional)
Config Param: PARQUET_COMPRESSION_CODEC_NAME
Archival Configs
Configurations that control archival.
Config Class
: org.apache.hudi.config.HoodieArchivalConfig
hoodie.archive.merge.small.file.limit.bytes
This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file.
Default Value: 20971520 (Optional)
Config Param: ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES
hoodie.keep.max.commits
Archiving service moves older entries from timeline into an archived log after each write, to keep the metadata overhead constant, even as the table size grows.This config controls the maximum number of instants to retain in the active timeline.
Default Value: 30 (Optional)
Config Param: MAX_COMMITS_TO_KEEP
hoodie.archive.merge.enable
When enable, hoodie will auto merge several small archive files into larger one. It's useful when storage scheme doesn't support append operation.
Default Value: false (Optional)
Config Param: ARCHIVE_MERGE_ENABLE
hoodie.archive.automatic
When enabled, the archival table service is invoked immediately after each commit, to archive commits if we cross a maximum value of commits. It's recommended to enable this, to ensure number of active commits is bounded.
Default Value: true (Optional)
Config Param: AUTO_ARCHIVE
hoodie.archive.delete.parallelism
Parallelism for deleting archived hoodie commits.
Default Value: 100 (Optional)
Config Param: DELETE_ARCHIVED_INSTANT_PARALLELISM_VALUE
hoodie.archive.beyond.savepoint
If enabled, archival will proceed beyond savepoint, skipping savepoint commits. If disabled, archival will stop at the earliest savepoint commit.
Default Value: false (Optional)
Config Param: ARCHIVE_BEYOND_SAVEPOINT
Since Version: 0.12.0
hoodie.commits.archival.batch
Archiving of instants is batched in best-effort manner, to pack more instants into a single archive log. This config controls such archival batch size.
Default Value: 10 (Optional)
Config Param: COMMITS_ARCHIVAL_BATCH_SIZE
hoodie.archive.async
Only applies when hoodie.archive.automatic is turned on. When turned on runs archiver async with writing, which can speed up overall write performance.
Default Value: false (Optional)
Config Param: ASYNC_ARCHIVE
Since Version: 0.11.0
hoodie.keep.min.commits
Similar to hoodie.keep.max.commits, but controls the minimum number ofinstants to retain in the active timeline.
Default Value: 20 (Optional)
Config Param: MIN_COMMITS_TO_KEEP
hoodie.archive.merge.files.batch.size
The number of small archive files to be merged at once.
Default Value: 10 (Optional)
Config Param: ARCHIVE_MERGE_FILES_BATCH_SIZE
Metadata Configs
Configurations used by the Hudi Metadata Table. This table maintains the metadata about a given Hudi table (e.g file listings) to avoid overhead of accessing cloud storage, during queries.
Config Class
: org.apache.hudi.common.config.HoodieMetadataConfig
hoodie.metadata.index.column.stats.parallelism
Parallelism to use, when generating column stats index.
Default Value: 10 (Optional)
Config Param: COLUMN_STATS_INDEX_PARALLELISM
Since Version: 0.11.0
hoodie.metadata.compact.max.delta.commits
Controls how often the metadata table is compacted.
Default Value: 10 (Optional)
Config Param: COMPACT_NUM_DELTA_COMMITS
Since Version: 0.7.0
hoodie.assume.date.partitioning
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
Default Value: false (Optional)
Config Param: ASSUME_DATE_PARTITIONING
Since Version: 0.3.0
hoodie.metadata.metrics.enable
Enable publishing of metrics around metadata table.
Default Value: false (Optional)
Config Param: METRICS_ENABLE
Since Version: 0.7.0
hoodie.metadata.index.bloom.filter.file.group.count
Metadata bloom filter index partition file group count. This controls the size of the base and log files and read parallelism in the bloom filter index partition. The recommendation is to size the file group count such that the base files are under 1GB.
Default Value: 4 (Optional)
Config Param: METADATA_INDEX_BLOOM_FILTER_FILE_GROUP_COUNT
Since Version: 0.11.0
_hoodie.metadata.ignore.spurious.deletes
There are cases when extra files are requested to be deleted from metadata table which are never added before. This config determines how to handle such spurious deletes
Default Value: true (Optional)
Config Param: IGNORE_SPURIOUS_DELETES
Since Version: 0.10.0
hoodie.file.listing.parallelism
Parallelism to use, when listing the table on lake storage.
Default Value: 200 (Optional)
Config Param: FILE_LISTING_PARALLELISM_VALUE
Since Version: 0.7.0
hoodie.metadata.index.async
Enable asynchronous indexing of metadata table.
Default Value: false (Optional)
Config Param: ASYNC_INDEX_ENABLE
Since Version: 0.11.0
hoodie.metadata.index.column.stats.column.list
Comma-separated list of columns for which column stats index will be built. If not set, all columns will be indexed
Default Value: N/A (Required)
Config Param: COLUMN_STATS_INDEX_FOR_COLUMNS
Since Version: 0.11.0
hoodie.metadata.enable.full.scan.log.files
Enable full scanning of log files while reading log records. If disabled, Hudi does look up of only interested entries.
Default Value: true (Optional)
Config Param: ENABLE_FULL_SCAN_LOG_FILES
Since Version: 0.10.0
hoodie.metadata.index.bloom.filter.enable
Enable indexing bloom filters of user data files under metadata table. When enabled, metadata table will have a partition to store the bloom filter index and will be used during the index lookups.
Default Value: false (Optional)
Config Param: ENABLE_METADATA_INDEX_BLOOM_FILTER
Since Version: 0.11.0
hoodie.metadata.clean.async
Enable asynchronous cleaning for metadata table
Default Value: false (Optional)
Config Param: ASYNC_CLEAN_ENABLE
Since Version: 0.7.0
hoodie.metadata.keep.max.commits
Similar to hoodie.metadata.keep.min.commits, this config controls the maximum number of instants to retain in the active timeline.
Default Value: 30 (Optional)
Config Param: MAX_COMMITS_TO_KEEP
Since Version: 0.7.0
hoodie.metadata.insert.parallelism
Parallelism to use when inserting to the metadata table
Default Value: 1 (Optional)
Config Param: INSERT_PARALLELISM_VALUE
Since Version: 0.7.0
hoodie.metadata.dir.filter.regex
Directories matching this regex, will be filtered out when initializing metadata table from lake storage for the first time.
Default Value: (Optional)
Config Param: DIR_FILTER_REGEX
Since Version: 0.7.0
hoodie.metadata.index.column.stats.processing.mode.override
By default Column Stats Index is automatically determining whether it should be read and processed either'in-memory' (w/in executing process) or using Spark (on a cluster), based on some factors like the size of the Index and how many columns are read. This config allows to override this behavior.
Default Value: N/A (Required)
Config Param: COLUMN_STATS_INDEX_PROCESSING_MODE_OVERRIDE
Since Version: 0.12.0
hoodie.metadata.keep.min.commits
Archiving service moves older entries from metadata table’s timeline into an archived log after each write, to keep the overhead constant, even as the metadata table size grows. This config controls the minimum number of instants to retain in the active timeline.
Default Value: 20 (Optional)
Config Param: MIN_COMMITS_TO_KEEP
Since Version: 0.7.0
hoodie.metadata.index.column.stats.inMemory.projection.threshold
When reading Column Stats Index, if the size of the expected resulting projection is below the in-memory threshold (counted by the # of rows), it will be attempted to be loaded "in-memory" (ie not using the execution engine like Spark, Flink, etc). If the value is above the threshold execution engine will be used to compose the projection.
Default Value: 100000 (Optional)
Config Param: COLUMN_STATS_INDEX_IN_MEMORY_PROJECTION_THRESHOLD
Since Version: 0.12.0
hoodie.metadata.index.column.stats.enable
Enable indexing column ranges of user data files under metadata table key lookups. When enabled, metadata table will have a partition to store the column ranges and will be used for pruning files during the index lookups.
Default Value: false (Optional)
Config Param: ENABLE_METADATA_INDEX_COLUMN_STATS
Since Version: 0.11.0
hoodie.metadata.index.bloom.filter.column.list
Comma-separated list of columns for which bloom filter index will be built. If not set, only record key will be indexed.
Default Value: N/A (Required)
Config Param: BLOOM_FILTER_INDEX_FOR_COLUMNS
Since Version: 0.11.0
hoodie.metadata.cleaner.commits.retained
Number of commits to retain, without cleaning, on metadata table.
Default Value: 3 (Optional)
Config Param: CLEANER_COMMITS_RETAINED
Since Version: 0.7.0
hoodie.metadata.index.check.timeout.seconds
After the async indexer has finished indexing upto the base instant, it will ensure that all inflight writers reliably write index updates as well. If this timeout expires, then the indexer will abort itself safely.
Default Value: 900 (Optional)
Config Param: METADATA_INDEX_CHECK_TIMEOUT_SECONDS
Since Version: 0.11.0
hoodie.metadata.populate.meta.fields
When enabled, populates all meta fields. When disabled, no meta fields are populated.
Default Value: false (Optional)
Config Param: POPULATE_META_FIELDS
Since Version: 0.10.0
hoodie.metadata.index.column.stats.file.group.count
Metadata column stats partition file group count. This controls the size of the base and log files and read parallelism in the column stats index partition. The recommendation is to size the file group count such that the base files are under 1GB.
Default Value: 2 (Optional)
Config Param: METADATA_INDEX_COLUMN_STATS_FILE_GROUP_COUNT
Since Version: 0.11.0
hoodie.metadata.enable
Enable the internal metadata table which serves table metadata like level file listings
Default Value: true (Optional)
Config Param: ENABLE
Since Version: 0.7.0
hoodie.metadata.index.bloom.filter.parallelism
Parallelism to use for generating bloom filter index in metadata table.
Default Value: 200 (Optional)
Config Param: BLOOM_FILTER_INDEX_PARALLELISM
Since Version: 0.11.0
Consistency Guard Configurations
The consistency guard related config options, to help talk to eventually consistent object storage.(Tip: S3 is NOT eventually consistent anymore!)
Config Class
: org.apache.hudi.common.fs.ConsistencyGuardConfig
hoodie.optimistic.consistency.guard.sleep_time_ms
Amount of time (in ms), to wait after which we assume storage is consistent.
Default Value: 500 (Optional)
Config Param: OPTIMISTIC_CONSISTENCY_GUARD_SLEEP_TIME_MS
Since Version: 0.6.0
hoodie.consistency.check.max_interval_ms
Maximum amount of time (in ms), to wait for consistency checking.
Default Value: 20000 (Optional)
Config Param: MAX_CHECK_INTERVAL_MS
Since Version: 0.5.0
Deprecated Version: 0.7.0
_hoodie.optimistic.consistency.guard.enable
Enable consistency guard, which optimistically assumes consistency is achieved after a certain time period.
Default Value: false (Optional)
Config Param: OPTIMISTIC_CONSISTENCY_GUARD_ENABLE
Since Version: 0.6.0
hoodie.consistency.check.enabled
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.
Default Value: false (Optional)
Config Param: ENABLE
Since Version: 0.5.0
Deprecated Version: 0.7.0
hoodie.consistency.check.max_checks
Maximum number of consistency checks to perform, with exponential backoff.
Default Value: 6 (Optional)
Config Param: MAX_CHECKS
Since Version: 0.5.0
Deprecated Version: 0.7.0
hoodie.consistency.check.initial_interval_ms
Amount of time (in ms) to wait, before checking for consistency after an operation on storage.
Default Value: 400 (Optional)
Config Param: INITIAL_CHECK_INTERVAL_MS
Since Version: 0.5.0
Deprecated Version: 0.7.0
FileSystem Guard Configurations
The filesystem retry related config options, to help deal with runtime exception like list/get/put/delete performance issues.
Config Class
: org.apache.hudi.common.fs.FileSystemRetryConfig
hoodie.filesystem.operation.retry.max_interval_ms
Maximum amount of time (in ms), to wait for next retry.
Default Value: 2000 (Optional)
Config Param: MAX_RETRY_INTERVAL_MS
Since Version: 0.11.0
hoodie.filesystem.operation.retry.enable
Enabled to handle list/get/delete etc file system performance issue.
Default Value: false (Optional)
Config Param: FILESYSTEM_RETRY_ENABLE
Since Version: 0.11.0
hoodie.filesystem.operation.retry.max_numbers
Maximum number of retry actions to perform, with exponential backoff.
Default Value: 4 (Optional)
Config Param: MAX_RETRY_NUMBERS
Since Version: 0.11.0
hoodie.filesystem.operation.retry.exceptions
The class name of the Exception that needs to be re-tryed, separated by commas. Default is empty which means retry all the IOException and RuntimeException from FileSystem
Default Value: (Optional)
Config Param: RETRY_EXCEPTIONS
Since Version: 0.11.0
hoodie.filesystem.operation.retry.initial_interval_ms
Amount of time (in ms) to wait, before retry to do operations on storage.
Default Value: 100 (Optional)
Config Param: INITIAL_RETRY_INTERVAL_MS
Since Version: 0.11.0
Write Configurations
Configurations that control write behavior on Hudi tables. These can be directly passed down from even higher level frameworks (e.g Spark datasources, Flink sink) and utilities (e.g DeltaStreamer).
Config Class
: org.apache.hudi.config.HoodieWriteConfig
hoodie.combine.before.upsert
When upserted records share same key, controls whether they should be first combined (i.e de-duplicated) before writing to storage. This should be turned off only if you are absolutely certain that there are no duplicates incoming, otherwise it can lead to duplicate keys and violate the uniqueness guarantees.
Default Value: true (Optional)
Config Param: COMBINE_BEFORE_UPSERT
hoodie.write.markers.type
Marker type to use. Two modes are supported: - DIRECT: individual marker file corresponding to each data file is directly created by the writer. - TIMELINE_SERVER_BASED: marker operations are all handled at the timeline service which serves as a proxy. New marker entries are batch processed and stored in a limited number of underlying files for efficiency. If HDFS is used or timeline server is disabled, DIRECT markers are used as fallback even if this is configure. For Spark structured streaming, this configuration does not take effect, i.e., DIRECT markers are always used for Spark structured streaming.
Default Value: TIMELINE_SERVER_BASED (Optional)
Config Param: MARKERS_TYPE
Since Version: 0.9.0
hoodie.consistency.check.max_interval_ms
Max time to wait between successive attempts at performing consistency checks
Default Value: 300000 (Optional)
Config Param: MAX_CONSISTENCY_CHECK_INTERVAL_MS
hoodie.embed.timeline.server.port
Port at which the timeline server listens for requests. When running embedded in each writer, it picks a free port and communicates to all the executors. This should rarely be changed.
Default Value: 0 (Optional)
Config Param: EMBEDDED_TIMELINE_SERVER_PORT_NUM
hoodie.auto.adjust.lock.configs
Auto adjust lock configurations when metadata table is enabled and for async table services.
Default Value: false (Optional)
Config Param: AUTO_ADJUST_LOCK_CONFIGS
Since Version: 0.11.0
hoodie.table.services.enabled
Master control to disable all table services including archive, clean, compact, cluster, etc.
Default Value: true (Optional)
Config Param: TABLE_SERVICES_ENABLED
Since Version: 0.11.0
hoodie.table.base.file.format
Base file format to store all the base file data.
Default Value: PARQUET (Optional)
Config Param: BASE_FILE_FORMAT
hoodie.avro.schema.validate
Validate the schema used for the write against the latest schema, for backwards compatibility.
Default Value: false (Optional)
Config Param: AVRO_SCHEMA_VALIDATE_ENABLE
hoodie.write.buffer.limit.bytes
Size of in-memory buffer used for parallelizing network reads and lake storage writes.
Default Value: 4194304 (Optional)
Config Param: WRITE_BUFFER_LIMIT_BYTES_VALUE
hoodie.insert.shuffle.parallelism
Parallelism for inserting records into the table. Inserts can shuffle data before writing to tune file sizes and optimize the storage layout.
Default Value: 200 (Optional)
Config Param: INSERT_PARALLELISM_VALUE
hoodie.embed.timeline.server.async
Controls whether or not, the requests to the timeline server are processed in asynchronous fashion, potentially improving throughput.
Default Value: false (Optional)
Config Param: EMBEDDED_TIMELINE_SERVER_USE_ASYNC_ENABLE
hoodie.rollback.parallelism
Parallelism for rollback of commits. Rollbacks perform delete of files or logging delete blocks to file groups on storage in parallel.
Default Value: 100 (Optional)
Config Param: ROLLBACK_PARALLELISM_VALUE
hoodie.write.status.storage.level
Write status objects hold metadata about a write (stats, errors), that is not yet committed to storage. This controls the how that information is cached for inspection by clients. We rarely expect this to be changed.
Default Value: MEMORY_AND_DISK_SER (Optional)
Config Param: WRITE_STATUS_STORAGE_LEVEL_VALUE
hoodie.writestatus.class
Subclass of org.apache.hudi.client.WriteStatus to be used to collect information about a write. Can be overridden to collection additional metrics/statistics about the data if needed.
Default Value: org.apache.hudi.client.WriteStatus (Optional)
Config Param: WRITE_STATUS_CLASS_NAME
hoodie.base.path
Base path on lake storage, under which all the table data is stored. Always prefix it explicitly with the storage scheme (e.g hdfs://, s3:// etc). Hudi stores all the main meta-data about commits, savepoints, cleaning audit logs etc in .hoodie directory under this base path directory.
Default Value: N/A (Required)
Config Param: BASE_PATH
hoodie.allow.empty.commit
Whether to allow generation of empty commits, even if no data was written in the commit. It's useful in cases where extra metadata needs to be published regardless e.g tracking source offsets when ingesting data
Default Value: true (Optional)
Config Param: ALLOW_EMPTY_COMMIT
hoodie.bulkinsert.user.defined.partitioner.class
If specified, this class will be used to re-partition records before they are bulk inserted. This can be used to sort, pack, cluster data optimally for common query patterns. For now we support a build-in user defined bulkinsert partitioner org.apache.hudi.execution.bulkinsert.RDDCustomColumnsSortPartitioner which can does sorting based on specified column values set by hoodie.bulkinsert.user.defined.partitioner.sort.columns
Default Value: N/A (Required)
Config Param: BULKINSERT_USER_DEFINED_PARTITIONER_CLASS_NAME
hoodie.table.name
Table name that will be used for registering with metastores like HMS. Needs to be same across runs.
Default Value: N/A (Required)
Config Param: TBL_NAME
hoodie.combine.before.delete
During delete operations, controls whether we should combine deletes (and potentially also upserts) before writing to storage.
Default Value: true (Optional)
Config Param: COMBINE_BEFORE_DELETE
hoodie.embed.timeline.server.threads
Number of threads to serve requests in the timeline server. By default, auto configured based on the number of underlying cores.
Default Value: -1 (Optional)
Config Param: EMBEDDED_TIMELINE_NUM_SERVER_THREADS
hoodie.fileid.prefix.provider.class
File Id Prefix provider class, that implements
org.apache.hudi.fileid.FileIdPrefixProvider
Default Value: org.apache.hudi.table.RandomFileIdPrefixProvider (Optional)
Config Param: FILEID_PREFIX_PROVIDER_CLASS
Since Version: 0.10.0
hoodie.fail.on.timeline.archiving
Timeline archiving removes older instants from the timeline, after each write operation, to minimize metadata overhead. Controls whether or not, the write should be failed as well, if such archiving fails.
Default Value: true (Optional)
Config Param: FAIL_ON_TIMELINE_ARCHIVING_ENABLE
hoodie.datasource.write.keygenerator.class
Key generator class, that implements
org.apache.hudi.keygen.KeyGenerator
extract a key out of incoming records.
Default Value: N/A (Required)
Config Param: KEYGENERATOR_CLASS_NAME
hoodie.combine.before.insert
When inserted records share same key, controls whether they should be first combined (i.e de-duplicated) before writing to storage.
Default Value: false (Optional)
Config Param: COMBINE_BEFORE_INSERT
hoodie.embed.timeline.server.gzip
Controls whether gzip compression is used, for large responses from the timeline server, to improve latency.
Default Value: true (Optional)
Config Param: EMBEDDED_TIMELINE_SERVER_COMPRESS_ENABLE
hoodie.markers.timeline_server_based.batch.interval_ms
The batch interval in milliseconds for marker creation batch processing
Default Value: 50 (Optional)
Config Param: MARKERS_TIMELINE_SERVER_BASED_BATCH_INTERVAL_MS
Since Version: 0.9.0
hoodie.skip.default.partition.validation
When table is upgraded from pre 0.12 to 0.12, we check for "default" partition and fail if found one. Users are expected to rewrite the data in those partitions. Enabling this config will bypass this validation
Default Value: false (Optional)
Config Param: SKIP_DEFAULT_PARTITION_VALIDATION
Since Version: 0.12.0
hoodie.markers.timeline_server_based.batch.num_threads
Number of threads to use for batch processing marker creation requests at the timeline server
Default Value: 20 (Optional)
Config Param: MARKERS_TIMELINE_SERVER_BASED_BATCH_NUM_THREADS
Since Version: 0.9.0
_.hoodie.allow.multi.write.on.same.instant
Default Value: false (Optional)
Config Param: ALLOW_MULTI_WRITE_ON_SAME_INSTANT_ENABLE
hoodie.datasource.write.payload.class
Payload class used. Override this, if you like to roll your own merge logic, when upserting/inserting. This will render any value set for PRECOMBINE_FIELD_OPT_VAL in-effective
Default Value: org.apache.hudi.common.model.OverwriteWithLatestAvroPayload (Optional)
Config Param: WRITE_PAYLOAD_CLASS_NAME
hoodie.bulkinsert.shuffle.parallelism
For large initial imports using bulk_insert operation, controls the parallelism to use for sort modes or custom partitioning donebefore writing records to the table.
Default Value: 200 (Optional)
Config Param: BULKINSERT_PARALLELISM_VALUE
hoodie.delete.shuffle.parallelism
Parallelism used for “delete” operation. Delete operations also performs shuffles, similar to upsert operation.
Default Value: 200 (Optional)
Config Param: DELETE_PARALLELISM_VALUE
hoodie.consistency.check.max_checks
Maximum number of checks, for consistency of written data.
Default Value: 7 (Optional)
Config Param: MAX_CONSISTENCY_CHECKS
hoodie.datasource.write.keygenerator.type
Easily configure one the built-in key generators, instead of specifying the key generator class.Currently supports SIMPLE, COMPLEX, TIMESTAMP, CUSTOM, NON_PARTITION, GLOBAL_DELETE Note This is being actively worked on. Please use
hoodie.datasource.write.keygenerator.class
instead.
Default Value: SIMPLE (Optional)
Config Param: KEYGENERATOR_TYPE
hoodie.merge.allow.duplicate.on.inserts
When enabled, we allow duplicate keys even if inserts are routed to merge with an existing file (for ensuring file sizing). This is only relevant for insert operation, since upsert, delete operations will ensure unique key constraints are maintained.
Default Value: false (Optional)
Config Param: MERGE_ALLOW_DUPLICATE_ON_INSERTS_ENABLE
hoodie.embed.timeline.server.reuse.enabled
Controls whether the timeline server instance should be cached and reused across the JVM (across task lifecycles)to avoid startup costs. This should rarely be changed.
Default Value: false (Optional)
Config Param: EMBEDDED_TIMELINE_SERVER_REUSE_ENABLED
hoodie.datasource.write.precombine.field
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(..)
Default Value: ts (Optional)
Config Param: PRECOMBINE_FIELD_NAME
hoodie.bulkinsert.sort.mode
Sorting modes to use for sorting records for bulk insert. This is use when user hoodie.bulkinsert.user.defined.partitioner.classis not configured. Available values are - GLOBAL_SORT: this ensures best file sizes, with lowest memory overhead at cost of sorting. PARTITION_SORT: Strikes a balance by only sorting within a partition, still keeping the memory overhead of writing lowest and best effort file sizing. NONE: No sorting. Fastest and matches
spark.write.parquet()
in terms of number of files, overheads
Default Value: NONE (Optional)
Config Param: BULK_INSERT_SORT_MODE
hoodie.avro.schema
Schema string representing the current write schema of the table. Hudi passes this to implementations of HoodieRecordPayload to convert incoming records to avro. This is also used as the write schema evolving records during an update.
Default Value: N/A (Required)
Config Param: AVRO_SCHEMA_STRING
hoodie.auto.commit
Controls whether a write operation should auto commit. This can be turned off to perform inspection of the uncommitted write before deciding to commit.
Default Value: true (Optional)
Config Param: AUTO_COMMIT_ENABLE
hoodie.embed.timeline.server
When true, spins up an instance of the timeline server (meta server that serves cached file listings, statistics),running on each writer's driver process, accepting requests during the write from executors.
Default Value: true (Optional)
Config Param: EMBEDDED_TIMELINE_SERVER_ENABLE
hoodie.timeline.layout.version
Controls the layout of the timeline. Version 0 relied on renames, Version 1 (default) models the timeline as an immutable log relying only on atomic writes for object storage.
Default Value: 1 (Optional)
Config Param: TIMELINE_LAYOUT_VERSION_NUM
Since Version: 0.5.1
hoodie.schema.cache.enable
cache query internalSchemas in driver/executor side
Default Value: false (Optional)
Config Param: ENABLE_INTERNAL_SCHEMA_CACHE
hoodie.upsert.shuffle.parallelism
Parallelism to use for upsert operation on the table. Upserts can shuffle data to perform index lookups, file sizing, bin packing records optimallyinto file groups.
Default Value: 200 (Optional)
Config Param: UPSERT_PARALLELISM_VALUE
hoodie.write.schema
The specified write schema. In most case, we do not need set this parameter, but for the case the write schema is not equal to the specified table schema, we can specify the write schema by this parameter. Used by MergeIntoHoodieTableCommand
Default Value: N/A (Required)
Config Param: WRITE_SCHEMA
hoodie.rollback.using.markers
Enables a more efficient mechanism for rollbacks based on the marker files generated during the writes. Turned on by default.
Default Value: true (Optional)
Config Param: ROLLBACK_USING_MARKERS_ENABLE
hoodie.merge.data.validation.enabled