Basic Configurations
This page covers the basic configurations you may use to write/read Hudi tables. This page only features a subset of the most frequently used configurations. For a full list of all configs, please visit the All Configurations page.
- Hudi Table Config: Basic Hudi Table configuration parameters.
- 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.
- Metastore and Catalog Sync Configs: Configurations used by the Hudi to sync metadata to external metastores and catalogs.
- Metrics Configs: These set of configs are used to enable monitoring and reporting of key Hudi stats and metrics.
- Kafka Connect Configs: These set of configs are used for Kafka Connect Sink Connector for writing Hudi Tables
- Hudi Streamer Configs: These set of configs are used for Hudi Streamer utility which provides the way to ingest from different sources such as DFS or Kafka.
In the tables below (N/A) means there is no default value set
Hudi Table Config
Basic Hudi Table configuration parameters.
Hudi Table Basic Configs
Configurations of the Hudi Table like type of ingestion, storage formats, hive table name etc. Configurations are loaded from hoodie.properties, these properties are usually set during initializing a path as hoodie base path and never changes during the lifetime of a hoodie table.
| Config Name | Default | Description |
|---|---|---|
| hoodie.bootstrap.base.path | (N/A) | Base path of the dataset that needs to be bootstrapped as a Hudi tableConfig Param: BOOTSTRAP_BASE_PATH |
| hoodie.database.name | (N/A) | 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 databaseConfig Param: DATABASE_NAME |
| hoodie.table.checksum | (N/A) | 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.Config Param: TABLE_CHECKSUMSince Version: 0.11.0 |
| hoodie.table.create.schema | (N/A) | Schema used when creating the table, for the first time.Config Param: CREATE_SCHEMA |
| hoodie.table.keygenerator.class | (N/A) | Key Generator class property for the hoodie tableConfig Param: KEY_GENERATOR_CLASS_NAME |
| hoodie.table.metadata.partitions | (N/A) | 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 readersConfig Param: TABLE_METADATA_PARTITIONSSince Version: 0.11.0 |
| hoodie.table.metadata.partitions.inflight | (N/A) | Comma-separated list of metadata partitions whose building is in progress. These partitions are not yet ready for use by the readers.Config Param: TABLE_METADATA_PARTITIONS_INFLIGHTSince Version: 0.11.0 |
| hoodie.table.name | (N/A) | Table name that will be used for registering with Hive. Needs to be same across runs.Config Param: NAME |
| hoodie.table.partition.fields | (N/A) | Fields used to partition the table. Concatenated values of these fields are used as the partition path, by invoking toString()Config Param: PARTITION_FIELDS |
| hoodie.table.precombine.field | (N/A) | 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.Config Param: PRECOMBINE_FIELD |
| hoodie.table.recordkey.fields | (N/A) | Columns used to uniquely identify the table. Concatenated values of these fields are used as the record key component of HoodieKey.Config Param: RECORDKEY_FIELDS |
| hoodie.table.secondary.indexes.metadata | (N/A) | The metadata of secondary indexesConfig Param: SECONDARY_INDEXES_METADATASince Version: 0.13.0 |
| hoodie.timeline.layout.version | (N/A) | Version of timeline used, by the table.Config Param: TIMELINE_LAYOUT_VERSION |
| hoodie.archivelog.folder | archived | path under the meta folder, to store archived timeline instants at.Config Param: ARCHIVELOG_FOLDER |
| hoodie.bootstrap.index.class | org.apache.hudi.common.bootstrap.index.hfile.HFileBootstrapIndex | Implementation to use, for mapping base files to bootstrap base file, that contain actual data.Config Param: BOOTSTRAP_INDEX_CLASS_NAME |
| hoodie.bootstrap.index.enable | true | Whether or not, this is a bootstrapped table, with bootstrap base data and an mapping index defined, default true.Config Param: BOOTSTRAP_INDEX_ENABLE |
| hoodie.compaction.payload.class | org.apache.hudi.common.model.OverwriteWithLatestAvroPayload | Payload class to use for performing compactions, i.e merge delta logs with current base file and then produce a new base file.Config Param: PAYLOAD_CLASS_NAME |
| hoodie.compaction.record.merger.strategy | eeb8d96f-b1e4-49fd-bbf8-28ac514178e5 | Id of merger strategy. Hudi will pick HoodieRecordMerger implementations in hoodie.datasource.write.record.merger.impls which has the same merger strategy idConfig Param: RECORD_MERGER_STRATEGYSince Version: 0.13.0 |
| hoodie.datasource.write.hive_style_partitioning | false | 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)Config Param: HIVE_STYLE_PARTITIONING_ENABLE |
| hoodie.partition.metafile.use.base.format | false | 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.Config Param: PARTITION_METAFILE_USE_BASE_FORMAT |
| hoodie.populate.meta.fields | true | 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 processingConfig Param: POPULATE_META_FIELDS |
| hoodie.table.base.file.format | PARQUET | Base file format to store all the base file data.Config Param: BASE_FILE_FORMAT |
| hoodie.table.cdc.enabled | false | When enable, persist the change data if necessary, and can be queried as a CDC query mode.Config Param: CDC_ENABLEDSince Version: 0.13.0 |
| hoodie.table.cdc.supplemental.logging.mode | DATA_BEFORE_AFTER | org.apache.hudi.common.table.cdc.HoodieCDCSupplementalLoggingMode: Change log capture supplemental logging mode. The supplemental log is used for accelerating the generation of change log details. OP_KEY_ONLY: Only keeping record keys in the supplemental logs, so the reader needs to figure out the update before image and after image. DATA_BEFORE: Keeping the before images in the supplemental logs, so the reader needs to figure out the update after images. DATA_BEFORE_AFTER(default): Keeping the before and after images in the supplemental logs, so the reader can generate the details directly from the logs.Config Param: CDC_SUPPLEMENTAL_LOGGING_MODESince Version: 0.13.0 |
| hoodie.table.log.file.format | HOODIE_LOG | Log format used for the delta logs.Config Param: LOG_FILE_FORMAT |
| hoodie.table.timeline.timezone | LOCAL | User can set hoodie commit timeline timezone, such as utc, local and so on. local is defaultConfig Param: TIMELINE_TIMEZONE |
| hoodie.table.type | COPY_ON_WRITE | The table type for the underlying data, for this write. This can’t change between writes.Config Param: TYPE |
| hoodie.table.version | ZERO | Version of table, used for running upgrade/downgrade steps between releases with potentially breaking/backwards compatible changes.Config Param: VERSION |
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 Name | Default | Description |
|---|---|---|
| hoodie.datasource.read.begin.instanttime | (N/A) | Required when hoodie.datasource.query.type is set to incremental. Represents the 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. Note that if hoodie.read.timeline.holes.resolution.policy set to USE_TRANSITION_TIME, will use instant's stateTransitionTime to perform comparison.Config Param: BEGIN_INSTANTTIME |
| hoodie.datasource.read.end.instanttime | (N/A) | Used when hoodie.datasource.query.type is set to incremental. Represents the instant time to limit incrementally fetched data to. When not specified latest commit time from timeline is assumed by default. When specified, new data written with an instant_time <= END_INSTANTTIME are fetched out. Point in time type queries make more sense with begin and end instant times specified. Note that if hoodie.read.timeline.holes.resolution.policy set to USE_TRANSITION_TIME, will use instant's stateTransitionTime to perform comparison.Config Param: END_INSTANTTIME |
| hoodie.datasource.query.type | snapshot | 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)Config Param: QUERY_TYPE |
| hoodie.datasource.write.precombine.field | ts | 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(..)Config Param: READ_PRE_COMBINE_FIELD |
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 Name | Default | Description |
|---|---|---|
| hoodie.datasource.hive_sync.mode | (N/A) | Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql.Config Param: HIVE_SYNC_MODE |
| hoodie.datasource.write.partitionpath.field | (N/A) | Partition path field. Value to be used at the partitionPath component of HoodieKey. Actual value obtained by invoking .toString()Config Param: PARTITIONPATH_FIELD |
| hoodie.datasource.write.recordkey.field | (N/A) | Record key field. Value to be used as the recordKey component of HoodieKey. Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using the dot notation eg: a.b.cConfig Param: RECORDKEY_FIELD |
| hoodie.clustering.async.enabled | false | Enable running of clustering service, asynchronously as inserts happen on the table.Config Param: ASYNC_CLUSTERING_ENABLESince Version: 0.7.0 |
| hoodie.clustering.inline | false | Turn on inline clustering - clustering will be run after each write operation is completeConfig Param: INLINE_CLUSTERING_ENABLESince Version: 0.7.0 |
| hoodie.datasource.hive_sync.enable | false | When set to true, register/sync the table to Apache Hive metastore.Config Param: HIVE_SYNC_ENABLED |
| hoodie.datasource.hive_sync.jdbcurl | jdbc:hive2://localhost:10000 | Hive metastore urlConfig Param: HIVE_URL |
| hoodie.datasource.hive_sync.metastore.uris | thrift://localhost:9083 | Hive metastore urlConfig Param: METASTORE_URIS |
| hoodie.datasource.meta.sync.enable | false | Enable Syncing the Hudi Table with an external meta store or data catalog.Config Param: META_SYNC_ENABLED |
| hoodie.datasource.write.hive_style_partitioning | false | 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)Config Param: HIVE_STYLE_PARTITIONING |
| hoodie.datasource.write.operation | upsert | Whether to do upsert, insert or bulk_insert for the write operation. Use bulk_insert 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.Config Param: OPERATION |
| hoodie.datasource.write.precombine.field | ts | 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(..)Config Param: PRECOMBINE_FIELD |
| hoodie.datasource.write.table.type | COPY_ON_WRITE | The table type for the underlying data, for this write. This can’t change between writes.Config Param: TABLE_TYPE |
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 Name | Default | Description |
|---|---|---|
| hoodie.database.name | (N/A) | Database name to register to Hive metastoreConfig Param: DATABASE_NAME |
| hoodie.table.name | (N/A) | Table name to register to Hive metastoreConfig Param: TABLE_NAME |
| path | (N/A) | 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 successfullyConfig Param: PATH |
| read.end-commit | (N/A) | End commit instant for reading, the commit time format should be 'yyyyMMddHHmmss'Config Param: READ_END_COMMIT |
| read.start-commit | (N/A) | Start commit instant for reading, the commit time format should be 'yyyyMMddHHmmss', by default reading from the latest instant for streaming readConfig Param: READ_START_COMMIT |
| archive.max_commits | 50 | Max number of commits to keep before archiving older commits into a sequential log, default 50Config Param: ARCHIVE_MAX_COMMITS |
| archive.min_commits | 40 | Min number of commits to keep before archiving older commits into a sequential log, default 40Config Param: ARCHIVE_MIN_COMMITS |
| cdc.enabled | false | When enable, persist the change data if necessary, and can be queried as a CDC query modeConfig Param: CDC_ENABLED |
| cdc.supplemental.logging.mode | DATA_BEFORE_AFTER | Setting 'op_key_only' persists the 'op' and the record key only, setting 'data_before' persists the additional 'before' image, and setting 'data_before_after' persists the additional 'before' and 'after' images.Config Param: SUPPLEMENTAL_LOGGING_MODE |
| changelog.enabled | false | 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 semanticsConfig Param: CHANGELOG_ENABLED |
| clean.async.enabled | true | Whether to cleanup the old commits immediately on new commits, enabled by defaultConfig Param: CLEAN_ASYNC_ENABLED |
| clean.retain_commits | 30 | 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 30Config Param: CLEAN_RETAIN_COMMITS |
| clustering.async.enabled | false | Async Clustering, default falseConfig Param: CLUSTERING_ASYNC_ENABLED |
| clustering.plan.strategy.small.file.limit | 600 | Files smaller than the size specified here are candidates for clustering, default 600 MBConfig Param: CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT |
| clustering.plan.strategy.target.file.max.bytes | 1073741824 | Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GBConfig Param: CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES |
| compaction.async.enabled | true | Async Compaction, enabled by default for MORConfig Param: COMPACTION_ASYNC_ENABLED |
| compaction.delta_commits | 5 | Max delta commits needed to trigger compaction, default 5 commitsConfig Param: COMPACTION_DELTA_COMMITS |
| hive_sync.enabled | false | Asynchronously sync Hive meta to HMS, default falseConfig Param: HIVE_SYNC_ENABLED |
| hive_sync.jdbc_url | jdbc:hive2://localhost:10000 | Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'Config Param: HIVE_SYNC_JDBC_URL |
| hive_sync.metastore.uris | Metastore uris for hive sync, default ''Config Param: HIVE_SYNC_METASTORE_URIS | |
| hive_sync.mode | HMS | Mode to choose for Hive ops. Valid values are hms, jdbc and hiveql, default 'hms'Config Param: HIVE_SYNC_MODE |
| hoodie.datasource.query.type | snapshot | 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: snapshotConfig Param: QUERY_TYPE |
| hoodie.datasource.write.hive_style_partitioning | false | 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)Config Param: HIVE_STYLE_PARTITIONING |
| hoodie.datasource.write.partitionpath.field | Partition path field. Value to be used at the partitionPath component of HoodieKey. Actual value obtained by invoking .toString(), default ''Config Param: PARTITION_PATH_FIELD | |
| hoodie.datasource.write.recordkey.field | uuid | Record key field. Value to be used as the recordKey component of HoodieKey. Actual value will be obtained by invoking .toString() on the field value. Nested fields can be specified using the dot notation eg: a.b.cConfig Param: RECORD_KEY_FIELD |
| index.type | FLINK_STATE | Index type of Flink write job, default is using state backed index.Config Param: INDEX_TYPE |
| metadata.compaction.delta_commits | 10 | Max delta commits for metadata table to trigger compaction, default 10Config Param: METADATA_COMPACTION_DELTA_COMMITS |
| metadata.enabled | false | Enable the internal metadata table which serves table metadata like level file listings, default disabledConfig Param: METADATA_ENABLED |
| precombine.field | ts | 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(..)Config Param: PRECOMBINE_FIELD |
| read.streaming.enabled | false | Whether to read as streaming source, default falseConfig Param: READ_AS_STREAMING |
| table.type | COPY_ON_WRITE | Type of table to write. COPY_ON_WRITE (or) MERGE_ON_READConfig Param: TABLE_TYPE |
| write.operation | upsert | The write operation, that this write should doConfig Param: OPERATION |
| write.parquet.max.file.size | 120 | 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.Config Param: WRITE_PARQUET_MAX_FILE_SIZE |
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.
Common Configurations
The following set of configurations are common across Hudi.
| Config Name | Default | Description |
|---|---|---|
| hoodie.base.path | (N/A) | 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.Config Param: BASE_PATH |
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 Name | Default | Description |
|---|---|---|
| hoodie.metadata.enable | true | Enable the internal metadata table which serves table metadata like level file listingsConfig Param: ENABLESince Version: 0.7.0 |
| hoodie.metadata.index.bloom.filter.enable | false | 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.Config Param: ENABLE_METADATA_INDEX_BLOOM_FILTERSince Version: 0.11.0 |
| hoodie.metadata.index.column.stats.enable | false | 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.Config Param: ENABLE_METADATA_INDEX_COLUMN_STATSSince Version: 0.11.0 |