Skip to main content
Version: 0.14.1

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.

  • 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.
note

In the tables below (N/A) means there is no default value set

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(...)

Basic Configs

Config NameDefaultDescription
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.typesnapshotWhether 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.fieldtsField 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(...)

Basic Configs

Config NameDefaultDescription
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.enabledfalseEnable running of clustering service, asynchronously as inserts happen on the table.
Config Param: ASYNC_CLUSTERING_ENABLE
Since Version: 0.7.0
hoodie.clustering.inlinefalseTurn on inline clustering - clustering will be run after each write operation is complete
Config Param: INLINE_CLUSTERING_ENABLE
Since Version: 0.7.0
hoodie.datasource.hive_sync.enablefalseWhen set to true, register/sync the table to Apache Hive metastore.
Config Param: HIVE_SYNC_ENABLED
hoodie.datasource.hive_sync.jdbcurljdbc:hive2://localhost:10000Hive metastore url
Config Param: HIVE_URL
hoodie.datasource.hive_sync.metastore.uristhrift://localhost:9083Hive metastore url
Config Param: METASTORE_URIS
hoodie.datasource.meta.sync.enablefalseEnable Syncing the Hudi Table with an external meta store or data catalog.
Config Param: META_SYNC_ENABLED
hoodie.datasource.write.hive_style_partitioningfalseFlag 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.operationupsertWhether 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.fieldtsField 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.typeCOPY_ON_WRITEThe table type for the underlying data, for this write. This can’t change between writes.
Config Param: TABLE_TYPE

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 jobs using the SQL can be configured through the options in WITH clause. The actual datasource level configs are listed below.

Basic Configs

Config NameDefaultDescription
hoodie.database.name(N/A)Database name to register to Hive metastore
Config Param: DATABASE_NAME
hoodie.table.name(N/A)Table name to register to Hive metastore
Config 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 successfully
Config 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 read
Config Param: READ_START_COMMIT
archive.max_commits50Max number of commits to keep before archiving older commits into a sequential log, default 50
Config Param: ARCHIVE_MAX_COMMITS
archive.min_commits40Min number of commits to keep before archiving older commits into a sequential log, default 40
Config Param: ARCHIVE_MIN_COMMITS
cdc.enabledfalseWhen enable, persist the change data if necessary, and can be queried as a CDC query mode
Config Param: CDC_ENABLED
cdc.supplemental.logging.modeDATA_BEFORE_AFTERSetting '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.enabledfalseWhether to keep all the intermediate changes, we try to keep all the changes of a record when enabled: 1). The sink accept the UPDATE_BEFORE message; 2). The source try to emit every changes of a record. The semantics is best effort because the compaction job would finally merge all changes of a record into one. default false to have UPSERT semantics
Config Param: CHANGELOG_ENABLED
clean.async.enabledtrueWhether to cleanup the old commits immediately on new commits, enabled by default
Config Param: CLEAN_ASYNC_ENABLED
clean.retain_commits30Number 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
Config Param: CLEAN_RETAIN_COMMITS
clustering.async.enabledfalseAsync Clustering, default false
Config Param: CLUSTERING_ASYNC_ENABLED
clustering.plan.strategy.small.file.limit600Files smaller than the size specified here are candidates for clustering, default 600 MB
Config Param: CLUSTERING_PLAN_STRATEGY_SMALL_FILE_LIMIT
clustering.plan.strategy.target.file.max.bytes1073741824Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups, default 1 GB
Config Param: CLUSTERING_PLAN_STRATEGY_TARGET_FILE_MAX_BYTES
compaction.async.enabledtrueAsync Compaction, enabled by default for MOR
Config Param: COMPACTION_ASYNC_ENABLED
compaction.delta_commits5Max delta commits needed to trigger compaction, default 5 commits
Config Param: COMPACTION_DELTA_COMMITS
hive_sync.enabledfalseAsynchronously sync Hive meta to HMS, default false
Config Param: HIVE_SYNC_ENABLED
hive_sync.jdbc_urljdbc:hive2://localhost:10000Jdbc URL for hive sync, default 'jdbc:hive2://localhost:10000'
Config Param: HIVE_SYNC_JDBC_URL
hive_sync.metastore.urisMetastore uris for hive sync, default ''
Config Param: HIVE_SYNC_METASTORE_URIS
hive_sync.modeHMSMode to choose for Hive ops. Valid values are hms, jdbc and hiveql, default 'hms'
Config Param: HIVE_SYNC_MODE
hoodie.datasource.query.typesnapshotDecides how data files need to be read, in 1) Snapshot mode (obtain latest view, based on row & columnar data); 2) incremental mode (new data since an instantTime); 3) Read Optimized mode (obtain latest view, based on columnar data) .Default: snapshot
Config Param: QUERY_TYPE
hoodie.datasource.write.hive_style_partitioningfalseWhether 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.fieldPartition 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.fielduuidRecord 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.typeFLINK_STATEIndex type of Flink write job, default is using state backed index.
Config Param: INDEX_TYPE
metadata.compaction.delta_commits10Max delta commits for metadata table to trigger compaction, default 10
Config Param: METADATA_COMPACTION_DELTA_COMMITS
metadata.enabledfalseEnable the internal metadata table which serves table metadata like level file listings, default disabled
Config Param: METADATA_ENABLED
precombine.fieldtsField 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.enabledfalseWhether to read as streaming source, default false
Config Param: READ_AS_STREAMING
table.typeCOPY_ON_WRITEType of table to write. COPY_ON_WRITE (or) MERGE_ON_READ
Config Param: TABLE_TYPE
write.operationupsertThe write operation, that this write should do
Config Param: OPERATION
write.parquet.max.file.size120Target 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.

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.

Basic Configs

Config NameDefaultDescription
hoodie.metadata.enabletrueEnable the internal metadata table which serves table metadata like level file listings
Config Param: ENABLE
Since Version: 0.7.0
hoodie.metadata.index.bloom.filter.enablefalseEnable 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.enablefalseEnable 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
hoodie.metadata.max.init.parallelism100000Maximum parallelism to use when initializing Record Index.
Config Param: RECORD_INDEX_MAX_PARALLELISM
Since Version: 0.14.0
hoodie.metadata.max.logfile.size2147483648Maximum size in bytes of a single log file. Larger log files can contain larger log blocks thereby reducing the number of blocks to search for keys
Config Param: MAX_LOG_FILE_SIZE_BYTES_PROP
Since Version: 0.14.0

Storage Configs

Configurations that control aspects around writing, sizing, reading base and log files.

Basic Configs

Config NameDefaultDescription
hoodie.parquet.compression.codecgzipCompression Codec for parquet files
Config Param: PARQUET_COMPRESSION_CODEC_NAME
hoodie.parquet.max.file.size125829120Target 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.

Basic Configs

Config NameDefaultDescription
hoodie.keep.max.commits30Archiving 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.commits20Similar 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.

Basic Configs

Config NameDefaultDescription
hoodie.bootstrap.base.path(N/A)Base path of the dataset that needs to be bootstrapped as a Hudi table
Config Param: BASE_PATH
Since Version: 0.6.0

Clean Configs

Cleaning (reclamation of older/unused file groups/slices).

Basic Configs

Config NameDefaultDescription
hoodie.clean.asyncfalseOnly 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.retained10Number 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.

Basic Configs

Config NameDefaultDescription
hoodie.clustering.async.enabledfalseEnable running of clustering service, asynchronously as inserts happen on the table.
Config Param: ASYNC_CLUSTERING_ENABLE
Since Version: 0.7.0
hoodie.clustering.inlinefalseTurn on inline clustering - clustering will be run after each write operation is complete
Config Param: INLINE_CLUSTERING
Since Version: 0.7.0
hoodie.clustering.plan.strategy.small.file.limit314572800Files smaller than the size in bytes specified here are candidates for clustering
Config Param: PLAN_STRATEGY_SMALL_FILE_LIMIT
Since Version: 0.7.0
hoodie.clustering.plan.strategy.target.file.max.bytes1073741824Each group can produce 'N' (CLUSTERING_MAX_GROUP_SIZE/CLUSTERING_TARGET_FILE_SIZE) output file groups
Config 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).

Basic Configs

Config NameDefaultDescription
hoodie.compact.inlinefalseWhen 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.commits5Number 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

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).

Basic Configs

Config NameDefaultDescription
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.fieldtsField 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.modeSINGLE_WRITERorg.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
hoodie.write.num.retries.on.conflict.failures0Maximum number of times to retry a batch on conflict failure.
Config Param: NUM_RETRIES_ON_CONFLICT_FAILURES
Since Version: 0.13.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

Basic Configs

Config NameDefaultDescription
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_partitioningfalseFlag 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

Basic Configs

Config NameDefaultDescription
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

Basic Configs

Config NameDefaultDescription
hoodie.datasource.meta.sync.enablefalseEnable Syncing the Hudi Table with an external meta store or data catalog.
Config Param: META_SYNC_ENABLED

BigQuery Sync Configs

Configurations used by the Hudi to sync metadata to Google BigQuery.

Basic Configs

Config NameDefaultDescription
hoodie.datasource.meta.sync.enablefalseEnable 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.

Basic Configs

Config NameDefaultDescription
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.enablefalseWhen set to true, register/sync the table to Apache Hive metastore.
Config Param: HIVE_SYNC_ENABLED
hoodie.datasource.hive_sync.jdbcurljdbc:hive2://localhost:10000Hive metastore url
Config Param: HIVE_URL
hoodie.datasource.hive_sync.metastore.uristhrift://localhost:9083Hive metastore url
Config Param: METASTORE_URIS
hoodie.datasource.meta.sync.enablefalseEnable 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.

Basic Configs

Config NameDefaultDescription
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.enablefalseWhen set to true, register/sync the table to Apache Hive metastore.
Config Param: HIVE_SYNC_ENABLED
hoodie.datasource.hive_sync.jdbcurljdbc:hive2://localhost:10000Hive metastore url
Config Param: HIVE_URL
hoodie.datasource.hive_sync.metastore.uristhrift://localhost:9083Hive metastore url
Config Param: METASTORE_URIS
hoodie.datasource.meta.sync.enablefalseEnable 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.

Basic Configs

Config NameDefaultDescription
hoodie.datasource.meta.sync.enablefalseEnable 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.

Basic Configs

Config NameDefaultDescription
hoodie.metrics.onfalseTurn on/off metrics reporting. off by default.
Config Param: TURN_METRICS_ON
Since Version: 0.5.0
hoodie.metrics.reporter.typeGRAPHITEType of metrics reporter.
Config Param: METRICS_REPORTER_TYPE_VALUE
Since Version: 0.5.0
hoodie.metricscompaction.log.blocks.onfalseTurn 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

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.

Basic Configs

Config NameDefaultDescription
bootstrap.serverslocalhost:9092The 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

Basic Configs

Config NameDefaultDescription
hoodie.streamer.source.kafka.topic(N/A)Kafka topic name. The config is specific to HoodieMultiTableStreamer
Config Param: KAFKA_TOPIC

Hudi Streamer SQL Transformer Configs

Configurations controlling the behavior of SQL transformer in Hudi Streamer.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.transformer.sql(N/A)SQL Query to be executed during write
Config Param: TRANSFORMER_SQL
hoodie.streamer.transformer.sql.file(N/A)File with a SQL script to be executed during write
Config 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.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.source.dfs.root(N/A)Root path of the source on DFS
Config 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.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.source.hoodieincr.path(N/A)Base-path for the source Hudi table
Config Param: HOODIE_SRC_BASE_PATH

Kafka Source Configs

Configurations controlling the behavior of Kafka source in Hudi Streamer.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.source.kafka.topic(N/A)Kafka topic name.
Config Param: KAFKA_TOPIC_NAME

Pulsar Source Configs

Configurations controlling the behavior of Pulsar source in Hudi Streamer.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.source.pulsar.topic(N/A)Name of the target Pulsar topic to source data from
Config Param: PULSAR_SOURCE_TOPIC_NAME
hoodie.streamer.source.pulsar.endpoint.admin.urlhttp://localhost:8080URL of the target Pulsar endpoint (of the form 'pulsar://host:port'
Config Param: PULSAR_SOURCE_ADMIN_ENDPOINT_URL
hoodie.streamer.source.pulsar.endpoint.service.urlpulsar://localhost:6650URL 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.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.s3.source.queue.url(N/A)Queue url for cloud object events
Config Param: S3_SOURCE_QUEUE_URL

SQL Source Configs

Configurations controlling the behavior of SQL source in Hudi Streamer.

Basic Configs

Config NameDefaultDescription
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

Basic Configs

Config NameDefaultDescription
hoodie.streamer.schemaprovider.registry.targetUrl(N/A)The schema of the target you are writing to e.g. https://foo:bar@schemaregistry.org
Config 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.org
Config Param: SRC_SCHEMA_REGISTRY_URL

File-based Schema Provider Configs

Configurations for file-based schema provider.

Basic Configs

Config NameDefaultDescription
hoodie.streamer.schemaprovider.source.schema.file(N/A)The schema of the source you are reading from
Config Param: SOURCE_SCHEMA_FILE
hoodie.streamer.schemaprovider.target.schema.file(N/A)The schema of the target you are writing to
Config Param: TARGET_SCHEMA_FILE