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