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_CHECKSUM Since 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_PARTITIONS Since 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_INFLIGHT Since 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_METADATA Since 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_STRATEGY Since 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_ENABLED Since 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_MODE Since 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.c Config Param: RECORDKEY_FIELD |
hoodie.clustering.async.enabled | false | Enable running of clustering service, asynchronously as inserts happen on the table.Config Param: ASYNC_CLUSTERING_ENABLE Since 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_ENABLE Since 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.c Config 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: ENABLE Since 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_FILTER Since 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_STATS Since Version: 0.11.0 |
Storage Configs
Configurations that control aspects around writing, sizing, reading base and log files.
Config Name | Default | Description |
---|---|---|
hoodie.parquet.compression.codec | gzip | Compression Codec for parquet filesConfig Param: PARQUET_COMPRESSION_CODEC_NAME |
hoodie.parquet.max.file.size | 125829120 | 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.Config Param: PARQUET_MAX_FILE_SIZE |
Archival Configs
Configurations that control archival.
Config Name | Default | Description |
---|---|---|
hoodie.keep.max.commits | 30 | 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. Config Param: MAX_COMMITS_TO_KEEP |
hoodie.keep.min.commits | 20 | Similar to hoodie.keep.max.commits, but controls the minimum number of instants to retain in the active timeline.Config Param: MIN_COMMITS_TO_KEEP |
Bootstrap Configs
Configurations that control how you want to bootstrap your existing tables for the first time into hudi. The bootstrap operation can flexibly avoid copying data over before you can use Hudi and support running the existing writers and new hudi writers in parallel, to validate the migration.
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: BASE_PATH Since Version: 0.6.0 |
Clean Configs
Cleaning (reclamation of older/unused file groups/slices).
Config Name | Default | Description |
---|---|---|
hoodie.clean.async | false | Only applies when hoodie.clean.automatic is turned on. When turned on runs cleaner async with writing, which can speed up overall write performance.Config Param: ASYNC_CLEAN |
hoodie.cleaner.commits.retained | 10 | When KEEP_LATEST_COMMITS cleaning policy is used, the 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.Config Param: CLEANER_COMMITS_RETAINED |
Clustering Configs
Configurations that control the clustering table service in hudi, which optimizes the storage layout for better query performance by sorting and sizing data files.
Config Name | Default | Description |
---|---|---|
hoodie.clustering.async.enabled | false | Enable running of clustering service, asynchronously as inserts happen on the table.Config Param: ASYNC_CLUSTERING_ENABLE Since 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 Since Version: 0.7.0 |
hoodie.clustering.plan.strategy.small.file.limit | 314572800 | Files smaller than the size in bytes specified here are candidates for clusteringConfig Param: PLAN_STRATEGY_SMALL_FILE_LIMIT Since Version: 0.7.0 |
hoodie.clustering.plan.strategy.target.file.max.bytes | 1073741824 | Each group can produce ‘N’ (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groupsConfig Param: PLAN_STRATEGY_TARGET_FILE_MAX_BYTES Since Version: 0.7.0 |
Compaction Configs
Configurations that control compaction (merging of log files onto a new base files).
Config Name | Default | Description |
---|---|---|
hoodie.compact.inline | false | When set to true, compaction service is triggered after each write. While being simpler operationally, this adds extra latency on the write path.Config Param: INLINE_COMPACT |
hoodie.compact.inline.max.delta.commits | 5 | Number of delta commits after the last compaction, before scheduling of a new compaction is attempted. This config takes effect only for the compaction triggering strategy based on the number of commits, i.e., NUM_COMMITS, NUM_COMMITS_AFTER_LAST_REQUEST, NUM_AND_TIME, and NUM_OR_TIME.Config Param: INLINE_COMPACT_NUM_DELTA_COMMITS |
Error table Configs
Configurations that are required for Error table configs
Config Name | Default | Description |
---|---|---|
hoodie.errortable.base.path | (N/A) | Base path for error table under which all error records would be stored.Config Param: ERROR_TABLE_BASE_PATH |
hoodie.errortable.target.table.name | (N/A) | Table name to be used for the error tableConfig Param: ERROR_TARGET_TABLE |
hoodie.errortable.write.class | (N/A) | Class which handles the error table writes. This config is used to configure a custom implementation for Error Table Writer. Specify the full class name of the custom error table writer as a value for this configConfig Param: ERROR_TABLE_WRITE_CLASS |
hoodie.errortable.enable | false | Config to enable error table. If the config is enabled, all the records with processing error in DeltaStreamer are transferred to error table.Config Param: ERROR_TABLE_ENABLED |
hoodie.errortable.insert.shuffle.parallelism | 200 | Config to set insert shuffle parallelism. The config is similar to hoodie.insert.shuffle.parallelism config but applies to the error table.Config Param: ERROR_TABLE_INSERT_PARALLELISM_VALUE |
hoodie.errortable.upsert.shuffle.parallelism | 200 | Config to set upsert shuffle parallelism. The config is similar to hoodie.upsert.shuffle.parallelism config but applies to the error table.Config Param: ERROR_TABLE_UPSERT_PARALLELISM_VALUE |
hoodie.errortable.validate.recordcreation.enable | true | Records that fail to be created due to keygeneration failure or other issues will be sent to the Error TableConfig Param: ERROR_ENABLE_VALIDATE_RECORD_CREATION Since Version: 0.15.0 |
hoodie.errortable.validate.targetschema.enable | false | Records with schema mismatch with Target Schema are sent to Error Table.Config Param: ERROR_ENABLE_VALIDATE_TARGET_SCHEMA |
hoodie.errortable.write.failure.strategy | ROLLBACK_COMMIT | The config specifies the failure strategy if error table write fails. Use one of - [ROLLBACK_COMMIT (Rollback the corresponding base table write commit for which the error events were triggered) , LOG_ERROR (Error is logged but the base table write succeeds) ]Config Param: ERROR_TABLE_WRITE_FAILURE_STRATEGY |
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 Hudi Streamer).
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 |
hoodie.table.name | (N/A) | Table name that will be used for registering with metastores like HMS. Needs to be same across runs.Config Param: TBL_NAME |
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_NAME |
hoodie.write.concurrency.mode | SINGLE_WRITER | org.apache.hudi.common.model.WriteConcurrencyMode: Concurrency modes for write operations. SINGLE_WRITER(default): Only one active writer to the table. Maximizes throughput. OPTIMISTIC_CONCURRENCY_CONTROL: Multiple writers can operate on the table with lazy conflict resolution using locks. This means that only one writer succeeds if multiple writers write to the same file group.Config Param: WRITE_CONCURRENCY_MODE |
Lock Configs
Configurations that control locking mechanisms required for concurrency control between writers to a Hudi table. Concurrency between Hudi’s own table services are auto managed internally.
Common Lock Configurations
Config Name | Default | Description |
---|---|---|
hoodie.write.lock.heartbeat_interval_ms | 60000 | Heartbeat interval in ms, to send a heartbeat to indicate that hive client holding locks.Config Param: LOCK_HEARTBEAT_INTERVAL_MS Since Version: 0.15.0 |
Key Generator Configs
Hudi maintains keys (record key + partition path) for uniquely identifying a particular record. These configs allow developers to setup the Key generator class that extracts these out of incoming records.
Key Generator Options
Config Name | Default | Description |
---|---|---|
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_NAME |
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.c Config Param: RECORDKEY_FIELD_NAME |
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 |
Index Configs
Configurations that control indexing behavior, which tags incoming records as either inserts or updates to older records.
Common Index Configs
Config Name | Default | Description |
---|---|---|
hoodie.index.type | (N/A) | org.apache.hudi.index.HoodieIndex$IndexType: Determines how input records are indexed, i.e., looked up based on the key for the location in the existing table. Default is SIMPLE on Spark engine, and INMEMORY on Flink and Java engines. HBASE: uses an external managed Apache HBase table to store record key to location mapping. HBase index is a global index, enforcing key uniqueness across all partitions in the table. INMEMORY: Uses in-memory hashmap in Spark and Java engine and Flink in-memory state in Flink for indexing. BLOOM: Employs bloom filters built out of the record keys, optionally also pruning candidate files using record key ranges. Key uniqueness is enforced inside partitions. GLOBAL_BLOOM: Employs bloom filters built out of the record keys, optionally also pruning candidate files using record key ranges. Key uniqueness is enforced across all partitions in the table. SIMPLE: Performs a lean join of the incoming update/delete records against keys extracted from the table on storage.Key uniqueness is enforced inside partitions. GLOBAL_SIMPLE: Performs a lean join of the incoming update/delete records against keys extracted from the table on storage.Key uniqueness is enforced across all partitions in the table. BUCKET: locates the file group containing the record fast by using bucket hashing, particularly beneficial in large scale. Use hoodie.index.bucket.engine to choose bucket engine type, i.e., how buckets are generated. FLINK_STATE: Internal Config for indexing based on Flink state. RECORD_INDEX: Index which saves the record key to location mappings in the HUDI Metadata Table. Record index is a global index, enforcing key uniqueness across all partitions in the table. Supports sharding to achieve very high scale.Config Param: INDEX_TYPE |
Metastore and Catalog Sync Configs
Configurations used by the Hudi to sync metadata to external metastores and catalogs.
Common Metadata Sync Configs
Config Name | Default | Description |
---|---|---|
hoodie.datasource.meta.sync.enable | false | Enable Syncing the Hudi Table with an external meta store or data catalog.Config Param: META_SYNC_ENABLED |
Glue catalog sync based client Configurations
Configs that control Glue catalog sync based client.
Config Name | Default | Description |
---|---|---|
hoodie.datasource.meta.sync.glue.partition_index_fields | Specify the partitions fields to index on aws glue. Separate the fields by semicolon. By default, when the feature is enabled, all the partition will be indexed. You can create up to three indexes, separate them by comma. Eg: col1;col2;col3,col2,col3Config Param: META_SYNC_PARTITION_INDEX_FIELDS Since Version: 0.15.0 |
|
hoodie.datasource.meta.sync.glue.partition_index_fields.enable | false | Enable aws glue partition index feature, to speedup partition based query patternConfig Param: META_SYNC_PARTITION_INDEX_FIELDS_ENABLE Since Version: 0.15.0 |
BigQuery Sync Configs
Configurations used by the Hudi to sync metadata to Google BigQuery.
Config Name | Default | Description |
---|---|---|
hoodie.datasource.meta.sync.enable | false | Enable Syncing the Hudi Table with an external meta store or data catalog.Config Param: META_SYNC_ENABLED |
Hive Sync Configs
Configurations used by the Hudi to sync metadata to Hive Metastore.
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.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 |
Global Hive Sync Configs
Global replication configurations used by the Hudi to sync metadata to Hive Metastore.
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.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 |
DataHub Sync Configs
Configurations used by the Hudi to sync metadata to DataHub.
Config Name | Default | Description |
---|---|---|
hoodie.datasource.meta.sync.enable | false | Enable Syncing the Hudi Table with an external meta store or data catalog.Config Param: META_SYNC_ENABLED |
Metrics Configs
These set of configs are used to enable monitoring and reporting of key Hudi stats and metrics.
Metrics Configurations
Enables reporting on Hudi metrics. Hudi publishes metrics on every commit, clean, rollback etc. The following sections list the supported reporters.
Config Name | Default | Description |
---|---|---|
hoodie.metrics.on | false | Turn on/off metrics reporting. off by default.Config Param: TURN_METRICS_ON Since Version: 0.5.0 |
hoodie.metrics.reporter.type | GRAPHITE | Type of metrics reporter.Config Param: METRICS_REPORTER_TYPE_VALUE Since Version: 0.5.0 |
hoodie.metricscompaction.log.blocks.on | false | Turn on/off metrics reporting for log blocks with compaction commit. off by default.Config Param: TURN_METRICS_COMPACTION_LOG_BLOCKS_ON Since Version: 0.14.0 |
Metrics Configurations for M3
Enables reporting on Hudi metrics using M3. Hudi publishes metrics on every commit, clean, rollback etc.
Config Name | Default | Description |
---|---|---|
hoodie.metrics.m3.env | production | M3 tag to label the environment (defaults to ‘production’), applied to all metrics.Config Param: M3_ENV Since Version: 0.15.0 |
hoodie.metrics.m3.host | localhost | M3 host to connect to.Config Param: M3_SERVER_HOST_NAME Since Version: 0.15.0 |
hoodie.metrics.m3.port | 9052 | M3 port to connect to.Config Param: M3_SERVER_PORT_NUM Since Version: 0.15.0 |
hoodie.metrics.m3.service | hoodie | M3 tag to label the service name (defaults to ‘hoodie’), applied to all metrics.Config Param: M3_SERVICE Since Version: 0.15.0 |
hoodie.metrics.m3.tags | Optional M3 tags applied to all metrics.Config Param: M3_TAGS Since Version: 0.15.0 |
Kafka Connect Configs
These set of configs are used for Kafka Connect Sink Connector for writing Hudi Tables
Kafka Sink Connect Configurations
Configurations for Kafka Connect Sink Connector for Hudi.
Config Name | Default | Description |
---|---|---|
bootstrap.servers | localhost:9092 | The bootstrap servers for the Kafka Cluster.Config Param: KAFKA_BOOTSTRAP_SERVERS |
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.
Hudi Streamer Configs
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.kafka.topic | (N/A) | Kafka topic name. The config is specific to HoodieMultiTableStreamerConfig Param: KAFKA_TOPIC |
Hudi Streamer SQL Transformer Configs
Configurations controlling the behavior of SQL transformer in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.transformer.sql | (N/A) | SQL Query to be executed during writeConfig Param: TRANSFORMER_SQL |
hoodie.streamer.transformer.sql.file | (N/A) | File with a SQL script to be executed during writeConfig Param: TRANSFORMER_SQL_FILE |
Hudi Streamer Source Configs
Configurations controlling the behavior of reading source data.
DFS Path Selector Configs
Configurations controlling the behavior of path selector for DFS source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.dfs.root | (N/A) | Root path of the source on DFSConfig Param: ROOT_INPUT_PATH |
Hudi Incremental Source Configs
Configurations controlling the behavior of incremental pulling from a Hudi table as a source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.hoodieincr.path | (N/A) | Base-path for the source Hudi tableConfig Param: HOODIE_SRC_BASE_PATH |
Kafka Source Configs
Configurations controlling the behavior of Kafka source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.kafka.topic | (N/A) | Kafka topic name.Config Param: KAFKA_TOPIC_NAME |
hoodie.streamer.source.kafka.proto.value.deserializer.class | org.apache.kafka.common.serialization.ByteArrayDeserializer | Kafka Proto Payload Deserializer ClassConfig Param: KAFKA_PROTO_VALUE_DESERIALIZER_CLASS Since Version: 0.15.0 |
Pulsar Source Configs
Configurations controlling the behavior of Pulsar source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.pulsar.topic | (N/A) | Name of the target Pulsar topic to source data fromConfig Param: PULSAR_SOURCE_TOPIC_NAME |
hoodie.streamer.source.pulsar.endpoint.admin.url | http://localhost:8080 | URL of the target Pulsar endpoint (of the form ‘pulsar://host:port’Config Param: PULSAR_SOURCE_ADMIN_ENDPOINT_URL |
hoodie.streamer.source.pulsar.endpoint.service.url | pulsar://localhost:6650 | URL of the target Pulsar endpoint (of the form ‘pulsar://host:port’Config Param: PULSAR_SOURCE_SERVICE_ENDPOINT_URL |
S3 Source Configs
Configurations controlling the behavior of S3 source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.s3.source.queue.url | (N/A) | Queue url for cloud object eventsConfig Param: S3_SOURCE_QUEUE_URL |
File-based SQL Source Configs
Configurations controlling the behavior of File-based SQL Source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.sql.file | (N/A) | SQL file path containing the SQL query to read source data.Config Param: SOURCE_SQL_FILE Since Version: 0.14.0 |
SQL Source Configs
Configurations controlling the behavior of SQL source in Hudi Streamer.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.source.sql.sql.query | (N/A) | SQL query for fetching source data.Config Param: SOURCE_SQL |
Hudi Streamer Schema Provider Configs
Configurations that control the schema provider for Hudi Streamer.
Hudi Streamer Schema Provider Configs
Config Name | Default | Description |
---|---|---|
hoodie.streamer.schemaprovider.registry.targetUrl | (N/A) | The schema of the target you are writing to e.g. https://foo:bar@schemaregistry.orgConfig Param: TARGET_SCHEMA_REGISTRY_URL |
hoodie.streamer.schemaprovider.registry.url | (N/A) | The schema of the source you are reading from e.g. https://foo:bar@schemaregistry.orgConfig Param: SRC_SCHEMA_REGISTRY_URL |
File-based Schema Provider Configs
Configurations for file-based schema provider.
Config Name | Default | Description |
---|---|---|
hoodie.streamer.schemaprovider.source.schema.file | (N/A) | The schema of the source you are reading fromConfig Param: SOURCE_SCHEMA_FILE |
hoodie.streamer.schemaprovider.target.schema.file | (N/A) | The schema of the target you are writing toConfig Param: TARGET_SCHEMA_FILE |