Hive Configuration Properties

This document describes the Hive user configuration properties (sometimes called parameters, variables, or options), and notes which releases introduced new properties.

The canonical list of configuration properties is managed in the HiveConf Java class, so refer to the HiveConf.java file for a complete list of configuration properties available in your Hive release.

For information about how to use these configuration properties, see Configuring Hive. That document also describes administrative configuration properties for setting up Hive in the Configuration Variables section. Hive Metastore Administration describes additional configuration properties for the metastore.

Version information

As of Hive 0.14.0 (HIVE-7211 ), a configuration name that starts with "hive." is regarded as a Hive system property. With the Configuration Properties#hive.conf.validation option true (default), any attempts to set a configuration property that starts with "hive." which is not registered to the Hive system will throw an exception.

Query and DDL Execution

hive.execution.engine
  • Default Value: mr (deprecated in Hive 2.0.0 – see below)
  • Added In: Hive 0.13.0 with HIVE-6103 and HIVE-6098

Chooses execution engine. Options are: mr (Map Reduce, default), tez (Tez execution, for Hadoop 2 only), or spark (Spark execution, for Hive 1.1.0 onward).

While mr remains the default engine for historical reasons, it is itself a historical engine and is deprecated in the Hive 2 line (HIVE-12300 ). It may be removed without further warning.

See Hive on Tez and Hive on Spark for more information, and see the Tez section and the Spark section below for their configuration properties.

hive.execution.mode
  • Chooses whether query fragments will run in container or in llap
  • Default Value: container
  • Valid settings
    • container: launch containers
    • llap: utilize llap nodes during execution of tasks
  • Added In: Hive 2.0 with HIVE-9460
mapred.reduce.tasks
  • Default Value: -1
  • Added In: Hive 0.1.0

The default number of reduce tasks per job. Typically set to a prime close to the number of available hosts. Ignored when mapred.job.tracker is "local". Hadoop set this to 1 by default, whereas Hive uses -1 as its default value. By setting this property to -1, Hive will automatically figure out what should be the number of reducers.

hive.exec.reducers.bytes.per.reducer
  • Default Value: 1,000,000,000 prior to Hive 0.14.0; 256 MB (256,000,000) in Hive 0.14.0 and later
  • Added In: Hive 0.2.0; default changed in 0.14.0 with HIVE-7158  (and HIVE-7917 )

Size per reducer. The default in Hive 0.14.0 and earlier is 1 GB, that is, if the input size is 10 GB then 10 reducers will be used. In Hive 0.14.0 and later the default is 256 MB, that is, if the input size is 1 GB then 4 reducers will be used.

hive.exec.reducers.max
  • Default Value: 999 prior to Hive 0.14.0; 1009 in Hive 0.14.0 and later
  • Added In: Hive 0.2.0; default changed in 0.14.0 with HIVE-7158 (and HIVE-7917 )

Maximum number of reducers that will be used. If the one specified in the configuration property Configuration Properties#mapred.reduce.tasks is negative, Hive will use this as the maximum number of reducers when automatically determining the number of reducers.

hive.jar.path
  • Default Value: (empty)
  • Added In: Hive 0.2.0 or earlier

The location of hive_cli.jar that is used when submitting jobs in a separate jvm.

hive.aux.jars.path
  • Default Value: (empty)
  • Added In: Hive 0.2.0 or earlier

The location of the plugin jars that contain implementations of user defined functions (UDFs) and SerDes.

hive.reloadable.aux.jars.path
  • Default Value: (empty)
  • Added In: Hive 0.14.0 with HIVE-7553

The locations of the plugin jars, which can be comma-separated folders or jars. They can be renewed (added, removed, or updated) by executing the Beeline reload command without having to restart HiveServer2. These jars can be used just like the auxiliary classes in hive.aux.jars.path  for creating UDFs or SerDes.

hive.exec.scratchdir
  • Default Value: /tmp/${user.name} in Hive 0.2.0 through 0.8.0; /tmp/hive-${user.name} in Hive 0.8.1 through 0.14.0; or /tmp/hive in Hive 0.14.0 and later
  • Added In: Hive 0.2.0; default changed in 0.8.1 and in 0.14.0 with HIVE-6847 and HIVE-8143

Scratch space for Hive jobs. This directory is used by Hive to store the plans for different map/reduce stages for the query as well as to stored the intermediate outputs of these stages.

Hive 0.14.0 and later:  HDFS root scratch directory for Hive jobs, which gets created with write all (733) permissionFor each connecting user, an HDFS scratch directory ${hive.exec.scratchdir}/<username> is created with ${Configuration Properties#hive.scratch.dir.permission }.

Also see hive.start.cleanup.scratchdir  and Configuration Properties#hive.scratchdir.lock .  When running Hive in local mode, see hive.exec.local.scratchdir .

hive.scratch.dir.permission
  • Default Value: 700
  • Added In: Hive 0.12.0 with HIVE-4487

The permission for the user-specific scratch directories that get created in the root scratch directory. (See hive.exec.scratchdir .)

hive.exec.local.scratchdir
  • Default Value: /tmp/${user.name}
  • Added In: Hive 0.10.0 with HIVE-1577

Scratch space for Hive jobs when Hive runs in local mode.  Also see hive.exec.scratchdir .

hive.hadoop.supports.splittable.combineinputformat

Whether to combine small input files so that fewer mappers are spawned.

hive.map.aggr
  • Default Value: true in Hive 0.3 and later; false in Hive 0.2
  • Added In: Hive 0.2.0

Whether to use map-side aggregation in Hive Group By queries.

hive.groupby.skewindata
  • Default Value: false
  • Added In: Hive 0.3.0

Whether there is skew in data to optimize group by queries.

hive.groupby.mapaggr.checkinterval
  • Default Value: 100000
  • Added In: Hive 0.3.0

Number of rows after which size of the grouping keys/aggregation classes is performed.

hive.new.job.grouping.set.cardinality
  • Default Value: 30
  • Added In: Hive 0.11.0 with HIVE-3552

Whether a new map-reduce job should be launched for grouping sets/rollups/cubes.

For a query like "select a, b, c, count(1) from T group by a, b, c with rollup;" four rows are created per row: (a, b, c), (a, b, null), (a, null, null), (null, null, null). This can lead to explosion across the map-reduce boundary if the cardinality of T is very high, and map-side aggregation does not do a very good job.

This parameter decides if Hive should add an additional map-reduce job. If the grouping set cardinality (4 in the example above) is more than this value, a new MR job is added under the assumption that the orginal "group by" will reduce the data size.

hive.mapred.local.mem
  • Default Value: 0
  • Added In: Hive 0.3.0

For local mode, memory of the mappers/reducers.

hive.map.aggr.hash.force.flush.memory.threshold
  • Default Value: 0.9
  • Added In: Hive 0.7.0 with HIVE-1830

The maximum memory to be used by map-side group aggregation hash table. If the memory usage is higher than this number, force to flush data.

hive.map.aggr.hash.percentmemory
  • Default Value: 0.5
  • Added In: Hive 0.2.0

Portion of total memory to be used by map-side group aggregation hash table.

hive.map.aggr.hash.min.reduction
  • Default Value: 0.5
  • Added In: Hive 0.4.0

Hash aggregation will be turned off if the ratio between hash table size and input rows is bigger than this number. Set to 1 to make sure hash aggregation is never turned off.

hive.optimize.groupby
  • Default Value: true
  • Added In: Hive 0.5.0

Whether to enable the bucketed group by from bucketed partitions/tables.

hive.optimize.countdistinct
  • Default Value: true
  • Added In: Hive 3.0.0 with HIVE-16654

Whether to rewrite count distinct into 2 stages, i.e., the first stage uses multiple reducers with the count distinct key and the second stage uses a single reducer without key.

hive.optimize.remove.sq_count_check
  • Default Value: false
  • Added In: Hive 3.0.0 with HIVE-16793

Whether to remove an extra join with sq_count_check UDF for scalar subqueries with constant group by keys. 

hive.multigroupby.singlemr

Whether to optimize multi group by query to generate a single M/R job plan. If the multi group by query has common group by keys, it will be optimized to generate a single M/R job. (This configuration property was removed in release 0.9.0.)

hive.multigroupby.singlereducer
  • Default Value: true
  • Added In: Hive 0.9.0 with HIVE-2621

Whether to optimize multi group by query to generate a single M/R  job plan. If the multi group by query has common group by keys, it will be optimized to generate a single M/R job.

hive.optimize.cp
  • Default Value: true
  • Added In: Hive 0.4.0 with HIVE-626
  • Removed In: Hive 0.13.0 with HIVE-4113

Whether to enable column pruner. (This configuration property was removed in release 0.13.0.)

hive.optimize.index.filter
  • Default Value: false
  • Added In: Hive 0.8.0 with HIVE-1644

Whether to enable automatic use of indexes.

Note:  See Configuration Properties#Indexing for more configuration properties related to Hive indexes.

hive.optimize.ppd
  • Default Value: true
  • Added In: Hive 0.4.0 with HIVE-279 , default changed to true in Hive 0.4.0 with HIVE-626

Whether to enable predicate pushdown (PPD). 

Note: Turn on Configuration Properties#hive.optimize.index.filter as well to use file format specific indexes with PPD.

hive.optimize.ppd.storage
  • Default Value: true
  • Added In: Hive 0.7.0

Whether to push predicates down into storage handlers. Ignored when hive.optimize.ppd is false.

hive.ppd.remove.duplicatefilters
  • Default Value: true
  • Added In: Hive 0.8.0

During query optimization, filters may be pushed down in the operator tree. If this config is true, only pushed down filters remain in the operator tree, and the original filter is removed. If this config is false, the original filter is also left in the operator tree at the original place.

hive.ppd.recognizetransivity
  • Default Value: true
  • Added In: Hive 0.8.1

Whether to transitively replicate predicate filters over equijoin conditions.

hive.join.emit.interval
  • Default Value: 1000
  • Added In: Hive 0.2.0

How many rows in the right-most join operand Hive should buffer before
emitting the join result.

hive.join.cache.size
  • Default Value: 25000
  • Added In: Hive 0.5.0

How many rows in the joining tables (except the streaming table)
should be cached in memory.

hive.mapjoin.bucket.cache.size

How many values in each key in the map-joined table should be cached in memory.

hive.mapjoin.followby.map.aggr.hash.percentmemory
  • Default Value: 0.3
  • Added In: Hive 0.7.0 with HIVE-1830

Portion of total memory to be used by map-side group aggregation hash table, when this group by is followed by map join.

hive.smalltable.filesize or hive.mapjoin.smalltable.filesize
  • Default Value: 25000000
  • Added In: Hive 0.7.0 with HIVE-1642 : hive.smalltable.filesize (replaced by hive.mapjoin.smalltable.filesize in Hive 0.8.1)
  • Added In: Hive 0.8.1 with HIVE-2499 : hive.mapjoin.smalltable.filesize

The threshold (in bytes) for the input file size of the small tables; if the file size is smaller than this threshold, it will try to convert the common join into map join.

hive.mapjoin.localtask.max.memory.usage

This number means how much memory the local task can take to hold the key/value into an in-memory hash table. If the local task's memory usage is more than this number, the local task will be aborted. It means the data of small table is too large to be held in memory.

hive.mapjoin.followby.gby.localtask.max.memory.usage
  • Default Value: 0.55
  • Added In: Hive 0.7.0 with HIVE-1830

This number means how much memory the local task can take to hold the key/value into an in-memory hash table when this map join is followed by a group by. If the local task's memory usage is more than this number, the local task will abort by itself. It means the data of the small table is too large to be held in memory.

hive.mapjoin.check.memory.rows

The number means after how many rows processed it needs to check the memory usage.

hive.ignore.mapjoin.hint
  • Default Value: true
  • Added In: Hive 0.11.0 with HIVE-4042

Whether Hive ignores the mapjoin hint.

hive.smbjoin.cache.rows

How many rows with the same key value should be cached in memory per sort-merge-bucket joined table.

hive.mapjoin.optimized.keys

Whether a MapJoin hashtable should use optimized (size-wise) keys, allowing the table to take less memory. Depending on the key, memory savings for the entire table can be 5-15% or so.

hive.mapjoin.optimized.hashtable
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-6430  

Whether Hive should use a memory-optimized hash table for MapJoin. Only works on Configuration Properties#Tez  and Configuration Properties#Spark , because memory-optimized hash table cannot be serialized. (Spark is supported starting from Hive 1.3.0, with HIVE-11180 .)

hive.mapjoin.optimized.hashtable.wbsize
  • Default Value: 10485760 (10 * 1024 * 1024)
  • Added In: Hive 0.14.0 with HIVE-6430  

Optimized hashtable (see Configuration Properties#hive.mapjoin.optimized.hashtable ) uses a chain of buffers to store data. This is one buffer size. Hashtable may be slightly faster if this is larger, but for small joins unnecessary memory will be allocated and then trimmed.

hive.mapjoin.lazy.hashtable

Whether a MapJoin hashtable should deserialize values on demand. Depending on how many values in the table the join will actually touch, it can save a lot of memory by not creating objects for rows that are not needed. If all rows are needed, obviously there's no gain.

hive.hashtable.initialCapacity
  • Default Value: 100000
  • Added In: Hive 0.7.0 with HIVE-1642

Initial capacity of mapjoin hashtable if statistics are absent, or if Configuration Properties#hive.hashtable.key.count.adjustment is set to 0.

hive.hashtable.key.count.adjustment
  • Default Value: 1.0
  • Added In: Hive 0.14.0 with HIVE-7616

Adjustment to mapjoin hashtable size derived from table and column statistics; the estimate of the number of keys is divided by this value. If the value is 0, statistics are not used and Configuration Properties#hive.hashtable.initialCapacity is used instead.

hive.hashtable.loadfactor
  • Default Value: 0.75
  • Added In: Hive 0.7.0 with HIVE-1642

In the process of Mapjoin, the key/value will be held in the hashtable. This value means the load factor for the in-memory hashtable.

hive.debug.localtask
  • Default Value: false
  • Added In: Hive 0.7.0 with HIVE-1642
hive.outerjoin.supports.filters
hive.optimize.skewjoin
  • Default Value: false
  • Added In: Hive 0.6.0

Whether to enable skew join optimization.  (Also see Configuration Properties#hive.optimize.skewjoin.compiletime .)

hive.skewjoin.key
  • Default Value: 100000
  • Added In: Hive 0.6.0

Determine if we get a skew key in join. If we see more than the specified number of rows with the same key in join operator, we think the key as a skew join key.

hive.skewjoin.mapjoin.map.tasks
  • Default Value: 10000
  • Added In: Hive 0.6.0

Determine the number of map task used in the follow up map join job for a skew join. It should be used together with Configuration Properties#hive.skewjoin.mapjoin.min.split to perform a fine grained control.

hive.skewjoin.mapjoin.min.split
  • Default Value: 33554432
  • Added In: Hive 0.6.0

Determine the number of map task at most used in the follow up map join job for a skew join by specifying the minimum split size. It should be used together with Configuration Properties#hive.skewjoin.mapjoin.map.tasks to perform a fine grained control.

hive.optimize.skewjoin.compiletime
  • Default Value: false
  • Added In: Hive 0.10.0

Whether to create a separate plan for skewed keys for the tables in the join. This is based on the skewed keys stored in the metadata. At compile time, the plan is broken into different joins: one for the skewed keys, and the other for the remaining keys. And then, a union is performed for the two joins generated above. So unless the same skewed key is present in both the joined tables, the join for the skewed key will be performed as a map-side join.

The main difference between this paramater and Configuration Properties#hive.optimize.skewjoin is that this parameter uses the skew information stored in the metastore to optimize the plan at compile time itself. If there is no skew information in the metadata, this parameter will not have any effect.
Both hive.optimize.skewjoin.compiletime and Configuration Properties#hive.optimize.skewjoin should be set to true. (Ideally, Configuration Properties#hive.optimize.skewjoin should be renamed as hive.optimize.skewjoin.runtime, but for backward compatibility that has not been done.)

If the skew information is correctly stored in the metadata, hive.optimize.skewjoin.compiletime will change the query plan to take care of it, and Configuration Properties#hive.optimize.skewjoin will be a no-op.

hive.optimize.union.remove
  • Default Value: false
  • Added In: Hive 0.10.0 with HIVE-3276

Whether to remove the union and push the operators between union and the filesink above union. This avoids an extra scan of the output by union. This is independently useful for union queries, and especially useful when Configuration Properties#hive.optimize.skewjoin.compiletime is set to true, since an extra union is inserted.

The merge is triggered if either of Configuration Properties#hive.merge.mapfiles or Configuration Properties#hive.merge.mapredfiles is set to true. If the user has set Configuration Properties#hive.merge.mapfiles to true and Configuration Properties#hive.merge.mapredfiles to false, the idea was that the number of reducers are few, so the number of files anyway is small. However, with this optimization, we are increasing the number of files possibly by a big margin. So, we merge aggresively.

hive.mapred.supports.subdirectories
  • Default Value: false
  • Added In: Hive 0.10.0 with HIVE-3276

Whether the version of Hadoop which is running supports sub-directories for tables/partitions. Many Hive optimizations can be applied if the Hadoop version supports sub-directories for tables/partitions. This support was added by MAPREDUCE-1501 .

hive.mapred.mode
  • Default Value: 
    • Hive 0.x: nonstrict
    • Hive 1.x: nonstrict
    • Hive 2.x: strict (HIVE-12413 )
  • Added In: Hive 0.3.0

The mode in which the Hive operations are being performed. In strict mode, some risky queries are not allowed to run. For example, full table scans are prevented (see HIVE-10454 ) and ORDER BY requires a LIMIT clause.

hive.exec.script.maxerrsize
  • Default Value: 100000
  • Added In: Hive 0.2.0

Maximum number of bytes a script is allowed to emit to standard error (per map-reduce task). This prevents runaway scripts from filling logs partitions to capacity.

hive.script.auto.progress
  • Default Value: false
  • Added In: Hive 0.4.0

Whether Hive Tranform/Map/Reduce Clause should automatically send progress information to TaskTracker to avoid the task getting killed because of inactivity. Hive sends progress information when the script is outputting to stderr. This option removes the need of periodically producing stderr messages, but users should be cautious because this may prevent infinite loops in the scripts to be killed by TaskTracker.

hive.exec.script.allow.partial.consumption
  • Default Value: false
  • Added In: Hive 0.5.0

When enabled, this option allows a user script to exit successfully without consuming all the data from the standard input.

hive.script.operator.id.env.var
  • Default Value: HIVE_SCRIPT_OPERATOR_ID
  • Added In: Hive 0.5.0

Name of the environment variable that holds the unique script operator ID in the user's transform function (the custom mapper/reducer that the user has specified in the query).

hive.script.operator.env.blacklist
  • Default Value: hive.txn.valid.txns,hive.script.operator.env.blacklist
  • Added In: Hive 0.14.0 with HIVE-8341

By default all values in the HiveConf object are converted to environment variables of the same name as the key (with '.' (dot) converted to '_' (underscore)) and set as part of the script operator's environment.  However, some values can grow large or are not amenable to translation to environment variables.  This value gives a comma separated list of configuration values that will not be set in the environment when calling a script operator.  By default the valid transaction list is excluded, as it can grow large and is sometimes compressed, which does not translate well to an environment variable.

Also see:
hive.exec.compress.output
  • Default Value: false
  • Added In: Hive 0.2.0

This controls whether the final outputs of a query (to a local/hdfs file or a Hive table) is compressed. The compression codec and other options are determined from Hadoop configuration variables mapred.output.compress* .

hive.exec.compress.intermediate
  • Default Value: false
  • Added In: Hive 0.2.0

This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed. The compression codec and other options are determined from Hadoop configuration variables mapred.output.compress*.

hive.exec.parallel
  • Default Value: false
  • Added In: Hive 0.5.0

Whether to execute jobs in parallel.  Applies to MapReduce jobs that can run in parallel, for example jobs processing different source tables before a join.  As of Hive 0.14 , also applies to move tasks that can run in parallel, for example moving files to insert targets during multi-insert.

hive.exec.parallel.thread.number
  • Default Value: 8
  • Added In: Hive 0.6.0

How many jobs at most can be executed in parallel.

hive.exec.rowoffset
  • Default Value: false
  • Added In: Hive 0.8.0

Whether to provide the row offset virtual column.

hive.task.progress
  • Default Value: false
  • Added In: Hive 0.5.0
  • Removed In: Hive 0.13.0 with HIVE-4518

Whether Hive should periodically update task progress counters during execution. Enabling this allows task progress to be monitored more closely in the job tracker, but may impose a performance penalty. This flag is automatically set to true for jobs with Configuration Properties#hive.exec.dynamic.partition set to true. (This configuration property was removed in release 0.13.0.)

hive.counters.group.name
  • Default Value: HIVE
  • Added In: Hive 0.13.0 with HIVE-4518

Counter group name for counters used during query execution. The counter group is used for internal Hive variables (CREATED_FILE, FATAL_ERROR, and so on).

hive.exec.pre.hooks
  • Default Value: (empty)
  • Added In: Hive 0.4.0

Comma-separated list of pre-execution hooks to be invoked for each statement. A pre-execution hook is specified as the name of a Java class which implements the org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.

hive.exec.post.hooks
  • Default Value: (empty)
  • Added In: Hive 0.5.0

Comma-separated list of post-execution hooks to be invoked for each statement. A post-execution hook is specified as the name of a Java class which implements the org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.

hive.exec.failure.hooks
  • Default Value: (empty)
  • Added In: Hive 0.8.0

Comma-separated list of on-failure hooks to be invoked for each statement. An on-failure hook is specified as the name of Java class which implements the org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface.

hive.merge.mapfiles
  • Default Value: true
  • Added In: Hive 0.4.0

Merge small files at the end of a map-only job.

hive.merge.mapredfiles
  • Default Value: false
  • Added In: Hive 0.4.0

Merge small files at the end of a map-reduce job.

hive.mergejob.maponly
  • Default Value: true
  • Added In: Hive 0.6.0
  • Removed In: Hive 0.11.0

Try to generate a map-only job for merging files if CombineHiveInputFormat is supported. (This configuration property was removed in release 0.11.0.)

hive.merge.size.per.task
  • Default Value: 256000000
  • Added In: Hive 0.4.0

Size of merged files at the end of the job.

hive.merge.smallfiles.avgsize
  • Default Value: 16000000
  • Added In: Hive 0.5.0

When the average output file size of a job is less than this number, Hive will start an additional map-reduce job to merge the output files into bigger files. This is only done for map-only jobs if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true.

hive.heartbeat.interval
  • Default Value: 1000
  • Added In: Hive 0.4.0

Send a heartbeat after this interval – used by mapjoin and filter operators.

hive.auto.convert.join
  • Default Value: false in 0.7.0 to 0.10.0; true in 0.11.0 and later (HIVE-3297 )  
  • Added In: 0.7.0 with HIVE-1642

Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. (Note that hive-default.xml.template incorrectly gives the default as false in Hive 0.11.0 through 0.13.1.)

hive.auto.convert.join.noconditionaltask

Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. If this parameter is on, and the sum of size for n-1 of the tables/partitions for an n-way join is smaller than the size specified by hive.auto.convert.join.noconditionaltask.size, the join is directly converted to a mapjoin (there is no conditional task).

hive.auto.convert.join.noconditionaltask.size
  • Default Value: 10000000
  • Added In: 0.11.0 with HIVE-3784

If hive.auto.convert.join.noconditionaltask is off, this parameter does not take effect. However, if it is on, and the sum of size for n-1 of the tables/partitions for an n-way join is smaller than this size, the join is directly converted to a mapjoin (there is no conditional task). The default is 10MB.

hive.auto.convert.join.use.nonstaged
  • Default Value: false
  • Added In: 0.13.0 with HIVE-6144  (default originally true, but changed to false with HIVE-6749 also in 0.13.0)

For conditional joins, if input stream from a small alias can be directly applied to the join operator without filtering or projection, the alias need not be pre-staged in the distributed cache via a mapred local task. Currently, this is not working with vectorization or Tez execution engine.

hive.merge.nway.joins
  • Default Value: true
  • Added In: 2.2.0 with HIVE-15655  

For multiple joins on the same condition, merge joins together into a single join operator. This is useful in the case of large shuffle joins to avoid a reshuffle phase. Disabling this in Tez will often provide a faster join algorithm in case of left outer joins or a general Snowflake schema.

hive.udtf.auto.progress
  • Default Value: false
  • Added In: Hive 0.5.0

Whether Hive should automatically send progress information to TaskTracker when using UDTF's to prevent the task getting killed because of inactivity. Users should be cautious because this may prevent TaskTracker from killing tasks with infinite loops.

hive.mapred.reduce.tasks.speculative.execution
  • Default Value: true
  • Added In: Hive 0.5.0

Whether speculative execution for reducers should be turned on.

hive.exec.counters.pull.interval
  • Default Value: 1000
  • Added In: Hive 0.6.0

The interval with which to poll the JobTracker for the counters the running job. The smaller it is the more load there will be on the jobtracker, the higher it is the less granular the caught will be.

hive.enforce.bucketing
  • Default Value: 
    • Hive 0.x: false
    • Hive 1.x: false
    • Hive 2.x: removed, which effectively makes it always true (HIVE-12331 )
  • Added In: Hive 0.6.0

Whether bucketing is enforced. If true, while inserting into the table, bucketing is enforced.

Set to true to support INSERT ... VALUES, UPDATE, and DELETE transactions in Hive 0.14.0 and 1.x.x. For a complete list of parameters required for turning on Hive transactions, see Configuration Properties#hive.txn.manager .

hive.enforce.sorting
  • Default Value: 
    • Hive 0.x: false
    • Hive 1.x: false
    • Hive 2.x: removed, which effectively makes it always true (HIVE-12331 )
  • Added In: Hive 0.6.0

Whether sorting is enforced. If true, while inserting into the table, sorting is enforced.

hive.optimize.bucketingsorting
  • Default Value: true
  • Added In: Hive 0.11.0 with HIVE-4240

If Configuration Properties#hive.enforce.bucketing or Configuration Properties#hive.enforce.sorting is true, don't create a reducer for enforcing bucketing/sorting for queries of the form:

insert overwrite table T2 select * from T1;

where T1 and T2 are bucketed/sorted by the same keys into the same number of buckets. (In Hive 2.0.0 and later, this parameter does not depend on Configuration Properties#hive.enforce.bucketing  or Configuration Properties#hive.enforce.sorting .)

hive.optimize.reducededuplication
  • Default Value: true
  • Added In: Hive 0.6.0

Remove extra map-reduce jobs if the data is already clustered by the same key which needs to be used again. This should always be set to true. Since it is a new feature, it has been made configurable.

hive.optimize.reducededuplication.min.reducer
  • Default Value: 4
  • Added In: Hive 0.11.0 with HIVE-2340

Reduce deduplication merges two RSs (reduce sink operators) by moving key/parts/reducer-num of the child RS to parent RS. That means if reducer-num of the child RS is fixed (order by or forced bucketing) and small, it can make very slow, single MR. The optimization will be disabled if number of reducers is less than specified value.

hive.optimize.correlation
  • Default Value: false
  • Added In: Hive 0.12.0 with HIVE-2206

Exploit intra-query correlations. For details see the Correlation Optimizer design document.

hive.optimize.limittranspose

Whether to push a limit through left/right outer join or union. If the value is true and the size of the outer input is reduced enough (as specified in hive.optimize.limittranspose.reductionpercentage and hive.optimize.limittranspose.reductiontuples ), the limit is pushed to the outer input or union; to remain semantically correct, the limit is kept on top of the join or the union too.

hive.optimize.limittranspose.reductionpercentage

When hive.optimize.limittranspose is true, this variable specifies the minimal percentage (fractional) reduction of the size of the outer input of the join or input of the union that the optimizer should get in order to apply the rule.

hive.optimize.limittranspose.reductiontuples

When hive.optimize.limittranspose is true, this variable specifies the minimal reduction in the number of tuples of the outer input of the join or input of the union that the optimizer should get in order to apply the rule.

hive.optimize.filter.stats.reduction
  • Default Value: false
  • Added In: Hive 2.1.0 with HIVE-13269

Whether to simplify comparison expressions in filter operators using column stats.

hive.optimize.sort.dynamic.partition
  • Default Value: true in Hive 0.13.0 and 0.13.1; false in Hive 0.14.0 and later (HIVE-8151 )
  • Added In: Hive 0.13.0 with HIVE-6455

When enabled, dynamic partitioning column will be globally sorted. This way we can keep only one record writer open for each partition value in the reducer thereby reducing the memory pressure on reducers.

hive.cbo.enable

When true, the cost based optimizer, which uses the Calcite framework, will be enabled.

hive.cbo.returnpath.hiveop

When true, this optimization to CBO Logical plan will add rule to introduce not null filtering on join keys.  Controls Calcite plan to Hive operator conversion.  Overrides hive.optimize.remove.identity.project when set to false.

hive.cbo.cnf.maxnodes
  • Default Value: -1
  • Added In: Hive 2.1.1 with HIVE-14021

When converting to conjunctive normal form (CNF), fail if the expression exceeds the specified threshold; the threshold is expressed in terms of the number of nodes (leaves and interior nodes). The default, -1, does not set up a threshold.

hive.optimize.null.scan
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-7385

When true, this optimization will try to not scan any rows from tables which can be determined at query compile time to not generate any rows (e.g., where 1 = 2, where false, limit 0 etc.).

hive.exec.dynamic.partition
  • Default Value: false prior to Hive 0.9.0; true in Hive 0.9.0 and later (HIVE-2835 )
  • Added In: Hive 0.6.0

Whether or not to allow dynamic partitions in DML/DDL.

hive.exec.dynamic.partition.mode
  • Default Value: strict
  • Added In: Hive 0.6.0

In strict mode, the user must specify at least one static partition in case the user accidentally overwrites all partitions. Inonstrict mode all partitions are allowed to be dynamic.

Set to nonstrict to support INSERT ... VALUES, UPDATE, and DELETE transactions (Hive 0.14.0 and later). For a complete list of parameters required for turning on Hive transactions, see Configuration Properties#hive.txn.manager .

hive.exec.max.dynamic.partitions
  • Default Value: 1000
  • Added In: Hive 0.6.0

Maximum number of dynamic partitions allowed to be created in total.

hive.exec.max.dynamic.partitions.pernode
  • Default Value: 100
  • Added In: Hive 0.6.0

Maximum number of dynamic partitions allowed to be created in each mapper/reducer node.

hive.exec.max.created.files
  • Default Value: 100000
  • Added In: Hive 0.7.0

Maximum number of HDFS files created by all mappers/reducers in a MapReduce job.

hive.exec.default.partition.name
  • Default Value: __HIVE_DEFAULT_PARTITION__
  • Added In: Hive 0.6.0

The default partition name in case the dynamic partition column value is null/empty string or any other values that cannot be escaped. This value must not contain any special character used in HDFS URI (e.g., ':', '%', '/' etc). The user has to be aware that the dynamic partition value should not contain this value to avoid confusions.

hive.fetch.output.serde
  • Default Value: org.apache.hadoop.hive.serde2.DelimitedJSONSerDe
  • Added In: Hive 0.7.0

The SerDe used by FetchTask to serialize the fetch output.

hive.exec.mode.local.auto
  • Default Value: false
  • Added In: Hive 0.7.0 with HIVE-1408

Lets Hive determine whether to run in local mode automatically.

hive.exec.mode.local.auto.inputbytes.max
  • Default Value: 134217728
  • Added In: Hive 0.7.0 with HIVE-1408

When Configuration Properties#hive.exec.mode.local.auto is true, input bytes should be less than this for local mode.

hive.exec.mode.local.auto.tasks.max

When Configuration Properties#hive.exec.mode.local.auto is true, the number of tasks should be less than this for local mode. Replaced in Hive 0.9.0 by Configuration Properties#hive.exec.mode.local.auto.input.files.max .

hive.exec.mode.local.auto.input.files.max
  • Default Value: 4
  • Added In: Hive 0.9.0 with HIVE-2651

When Configuration Properties#hive.exec.mode.local.auto is true, the number of tasks should be less than this for local mode.

hive.exec.drop.ignorenonexistent

Do not report an error if DROP TABLE/VIEW/PARTITION/INDEX/TEMPORARY FUNCTION specifies a non-existent table/view. Also applies to permanent functions as of Hive 0.13.0.

hive.exec.show.job.failure.debug.info
  • Default Value: true
  • Added In: Hive 0.7.0

If a job fails, whether to provide a link in the CLI to the task with the most failures, along with debugging hints if applicable.

hive.auto.progress.timeout
  • Default Value: 0
  • Added In: Hive 0.7.0

How long to run autoprogressor for the script/UDTF operators (in seconds). Set to 0 for forever.

hive.table.parameters.default
  • Default Value: (empty)
  • Added In: Hive 0.7.0

Default property values for newly created tables.

hive.variable.substitute
  • Default Value: true
  • Added In: Hive 0.7.0

This enables substitution using syntax like ${var} ${system:var} and ${env:var}.

hive.error.on.empty.partition
  • Default Value: false
  • Added In: Hive 0.7.0

Whether to throw an exception if dynamic partition insert generates empty results.

hive.exim.uri.scheme.whitelist
  • Default Value: hdfs,pfile prior to Hive 2.2.0; hdfs,pfile,file in Hive 2.2.0 and later

  • Added In: Hive 0.8.0 with HIVE-1918 ; default changed in Hive 2.2.0 with HIVE-15151

A comma separated list of acceptable URI schemes for import and export.

hive.limit.row.max.size
  • Default Value: 100000
  • Added In: Hive 0.8.0

When trying a smaller subset of data for simple LIMIT, how much size we need to guarantee each row to have at least.

hive.limit.optimize.limit.file
  • Default Value: 10
  • Added In: Hive 0.8.0

When trying a smaller subset of data for simple LIMIT, maximum number of files we can sample.

hive.limit.optimize.enable
  • Default Value: false
  • Added In: Hive 0.8.0

Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first.

hive.limit.optimize.fetch.max
  • Default Value: 50000
  • Added In: Hive 0.8.0

Maximum number of rows allowed for a smaller subset of data for simple LIMIT, if it is a fetch query. Insert queries are not restricted by this limit.

hive.rework.mapredwork
  • Default Value: false
  • Added In: Hive 0.8.0

Should rework the mapred work or not. This is first introduced by SymlinkTextInputFormat to replace symlink files with real paths at compile time.

hive.sample.seednumber
  • Default Value: 0
  • Added In: Hive 0.8.0

A number used to percentage sampling. By changing this number, user will change the subsets of data sampled.

hive.autogen.columnalias.prefix.label
  • Default Value: _c
  • Added In: Hive 0.8.0

String used as a prefix when auto generating column alias. By default the prefix label will be appended with a column position number to form the column alias. Auto generation would happen if an aggregate function is used in a select clause without an explicit alias.

hive.autogen.columnalias.prefix.includefuncname
  • Default Value: false
  • Added In: Hive 0.8.0

Whether to include function name in the column alias auto generated by Hive.

hive.exec.perf.logger
  • Default Value: org.apache.hadoop.hive.ql.log.PerfLogger
  • Added In: Hive 0.8.0

The class responsible logging client side performance metrics. Must be a subclass of org.apache.hadoop.hive.ql.log.PerfLogger.

hive.start.cleanup.scratchdir

To clean up the Hive scratch directory while starting the Hive server (or HiveServer2). This is not an option for a multi-user environment since it will accidentally remove the scratch directory in use.

hive.scratchdir.lock
  • Default Value: false
  • Added In: Hive 1.3.0 and 2.1.0 (but not 2.0.x) with HIVE-13429

When true, holds a lock file in the scratch directory. If a Hive process dies and accidentally leaves a dangling scratchdir behind, the cleardanglingscratchdir tool will remove it.

When false, does not create a lock file and therefore the cleardanglingscratchdir tool cannot remove any dangling scratch directories.

hive.output.file.extension
  • Default Value: (empty)
  • Added In: Hive 0.8.1

String used as a file extension for output files. If not set, defaults to the codec extension for text files (e.g. ".gz"), or no extension otherwise.

hive.insert.into.multilevel.dirs
  • Default Value: false
  • Added In: Hive 0.8.1

Whether to insert into multilevel nested directories like "insert directory '/HIVEFT25686/chinna/' from table".

The following error may be shown when inserting into a nested directory that does not exist:
ERROR org.apache.hadoop.hive.ql.exec.Task: Failed with exception Unable to rename: <xxxx>

To enable automatic subdirectory generation set 'hive.insert.into.multilevel.dirs=true'

hive.conf.validation
  • Default Value: true
  • Added In: Hive 0.10.0 with HIVE-2848

Enables type checking for registered Hive configurations.

As of Hive 0.14.0 (HIVE-7211 ), a configuration name that starts with "hive." is regarded as a Hive system property. With hive.conf.validation true (default), any attempts to set a configuration property that starts with "hive." which is not registered to the Hive system will throw an exception.

hive.fetch.task.conversion
  • Default Value: minimal in Hive 0.10.0 through 0.13.1, more in Hive 0.14.0 and later
  • Added In: Hive 0.10.0 with HIVE-2925 ; default changed in Hive 0.14.0 with HIVE-7397

Some select queries can be converted to a single FETCH task, minimizing latency. Currently the query should be single sourced not having any subquery and should not have any aggregations or distincts (which incur RS – ReduceSinkOperator, requiring a MapReduce task), lateral views and joins.

Supported values are none, minimal and more.

0. none:  Disable hive.fetch.task.conversion (value added in Hive 0.14.0 with HIVE-8389 )
1. minimal:  SELECT *, FILTER on partition columns (WHERE and HAVING clauses), LIMIT only
2. more:  SELECT, FILTER, LIMIT only (including TABLESAMPLE, virtual columns)

"more" can take any kind of expressions in the SELECT clause, including UDFs.
(UDTFs and lateral views are not yet supported – see HIVE-5718 .)

hive.map.groupby.sorted
  • Default Value:
    • Hive 0.x and 1.x: false
    • Hive 2.0 and later: true (HIVE-12325 )
  • Added In: Hive 0.10.0 with HIVE-3432

If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform the group by in the mapper by using BucketizedHiveInputFormat. The only downside to this is that it limits the number of mappers to the number of files.

hive.map.groupby.sorted.testmode

If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform the group by in the mapper by using BucketizedHiveInputFormat. If the test mode is set, the plan is not converted, but a query property is set to denote the same. (This configuration property was removed in release 2.0.0.)

hive.groupby.orderby.position.alias
  • Default Value: false
  • Added In: Hive 0.11.0 with HIVE-581
  • Deprecated In: Hive 2.2.0 with HIVE-15797

Whether to enable using Column Position Alias in GROUP BY and ORDER BY clauses of queries (deprecated as of Hive 2.2.0; use Configuration Properties#hive.groupby.position.alias and Configuration Properties#hive.orderby.position.alias instead).

hive.groupby.position.alias
  • Default Value: false
  • Added In: Hive 2.2.0 with HIVE-15797

Whether to enable using Column Position Alias in GROUP BY.

hive.orderby.position.alias
  • Default Value: true
  • Added In: Hive 2.2.0 with HIVE-15797

Whether to enable using Column Position Alias in ORDER BY.

hive.fetch.task.aggr
  • Default Value: false
  • Added In: Hive 0.12.0 with HIVE-4002 (description added in Hive 0.13.0 with HIVE-5793 )

Aggregation queries with no group-by clause (for example, select count(*) from src) execute final aggregations in a single reduce task. If this parameter is set to true, Hive delegates the final aggregation stage to a fetch task, possibly decreasing the query time.

hive.fetch.task.conversion.threshold
  • Default Value: -1 in Hive 0.13.0 and 0.13.1, 1073741824 (1 GB) in Hive 0.14.0 and later 
  • Added In: Hive 0.13.0 with HIVE-3990 ; default changed in Hive 0.14.0 with HIVE-7397

Input threshold (in bytes) for applying hive.fetch.task.conversion . If target table is native, input length is calculated by summation of file lengths. If it's not native, the storage handler for the table can optionally implement the org.apache.hadoop.hive.ql.metadata.InputEstimator interface. A negative threshold means hive.fetch.task.conversion  is applied without any input length threshold.

hive.limit.pushdown.memory.usage
  • Default Value: -1
  • Added In: Hive 0.12.0 with HIVE-3562

The maximum memory to be used for hash in RS operator for top K selection. The default value "-1" means no limit.

hive.cache.expr.evaluation
  • Default Value: true
  • Added In: Hive 0.12.0 with HIVE-4209
  • Bug Fix: Hive 0.14.0 with HIVE-7314 (expression caching doesn't work when using UDF inside another UDF or a Hive function)

If true, the evaluation result of a deterministic expression referenced twice or more will be cached. For example, in a filter condition like "... where key + 10 > 10 or key + 10 = 0" the expression "key + 10" will be evaluated/cached once and reused for the following expression ("key + 10 = 0"). Currently, this is applied only to expressions in select or filter operators.

hive.resultset.use.unique.column.names
  • Default Value: true
  • Added In: Hive 0.13.0 with HIVE-6687

Make column names unique in the result set by qualifying column names with table alias if needed. Table alias will be added to column names for queries of type "select *" or if query explicitly uses table alias "select r1.x..".

hive.support.quoted.identifiers
  • Default Value: column
  • Added In: Hive 0.13.0 with HIVE-6013

Whether to use quoted identifiers.  Value can be "none" or "column".

column:  Column names can contain any Unicode character. Any column name that is specified within backticks (`) is treated literally. Within a backtick string, use double backticks (``) to represent a backtick character.
none:  Only alphanumeric and underscore characters are valid in identifiers. Backticked names are interpreted as regular expressions. This is also the behavior in releases prior to 0.13.0.

hive.plan.serialization.format
  • Default Value: kryo
  • Added In: Hive 0.13.0 with HIVE-1511
  • Removed a Value In: Hive 2.0.0 with HIVE-12609 javaXML is no longer supported

Query plan format serialization between client and task nodes. Two supported values are kryo and javaXML (prior to Hive 2.0.0). Kryo is the default (and starting from Hive 2.0.0 Kryo is the only supported value).

hive.exec.check.crossproducts
  • Default Value: true
  • Added In: Hive 0.13.0 with HIVE-6643

Check if a query plan contains a cross product. If there is one, output a warning to the session's console.

hive.display.partition.cols.separately
  • Default Value: true
  • Added In: Hive 0.13.0 with HIVE-6689

In older Hive versions (0.10 and earlier) no distinction was made between partition columns or non-partition columns while displaying columns in DESCRIBE TABLE. From version 0.12 onwards, they are displayed separately. This flag will let you get the old behavior, if desired. See test-case in patch for HIVE-6689 .

hive.limit.query.max.table.partition

To protect the cluster, this controls how many partitions can be scanned for each partitioned table. The default value "-1" means no limit. The limit on partitions does not affect metadata-only queries.

hive.files.umask.value

Obsolete:  The dfs.umask value for the Hive-created folders.

hive.optimize.sampling.orderby
  • Default Value: false
  • Added In: Hive 0.12.0 with HIVE-1402

Uses sampling on order-by clause for parallel execution.

hive.optimize.sampling.orderby.number
  • Default Value: 1000
  • Added In: Hive 0.12.0 with HIVE-1402

With hive.optimize.sampling.orderby=true, total number of samples to be obtained to calculate partition keys.

hive.optimize.sampling.orderby.percent
  • Default Value: 0.1
  • Added In: Hive 0.12.0 with HIVE-1402

With hive.optimize.sampling.orderby=true, probability with which a row will be chosen.

hive.compat
  • Default Value: 0.12
  • Added In: Hive 0.13.0 with HIVE-6012

Enable (configurable) deprecated behaviors of arithmetic operations by setting the desired level of backward compatibility. The default value gives backward-compatible return types for numeric operations. Other supported release numbers give newer behavior for numeric operations, for example 0.13 gives the more SQL compliant return types introduced in HIVE-5356 .

The value "latest" specifies the latest supported level. Currently, this only affects division of integers.

Setting to 0.12 (default) maintains division behavior in Hive 0.12 and earlier releases: int / int = double.
Setting to 0.13 gives division behavior in Hive 0.13 and later releases: int / int = decimal.

An invalid setting will cause an error message, and the default support level will be used.

hive.optimize.constant.propagation
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-5771

Whether to enable the constant propagation optimizer.

hive.entity.capture.transform

Enable capturing compiler read entity of transform URI which can be introspected in the semantic and exec hooks.

hive.support.sql11.reserved.keywords
  • Default Value: true
  • Added In: Hive 1.2.0 with HIVE-6617

Whether to enable support for SQL2011 reserved keywords. When enabled, will support (part of) SQL2011 reserved keywords.

hive.log.explain.output

When enabled, will log EXPLAIN EXTENDED output for the query at log4j INFO level and in HiveServer2 WebUI / Drilldown / Query Plan.

From Hive 3.1.0 onwards, this configuration property only logs to the log4j INFO. To log the EXPLAIN EXTENDED output in WebUI / Drilldown / Query Plan from Hive 3.1.0 onwards, use Configuration Properties#hive.server2.webui.explain.output . 

hive.explain.user
  • Default Value: false
  • Added In: Hive 1.2.0 with HIVE-9780

Whether to show explain result at user levelWhen enabled, will log EXPLAIN output for the query at user level. (Tez only.  For Spark, see hive.spark.explain.user .)

hive.typecheck.on.insert
  • Default Value: true
  • Added In: Hive 0.12.0 with HIVE-5297 for insert partition
  • Extended In: Hive 1.2 with HIVE-10307 for alter, describe partition, etc.

Whether to check, convert, and normalize partition value specified in partition specification to conform to the partition column type.

hive.exec.temporary.table.storage
  • Default Value: default

  • Added In: Hive 1.1.0 with HIVE-7313

Expects one of [memory, ssd, default].

Define the storage policy for temporary tables. Choices between memory, ssd and default. See HDFS Storage Types and Storage Policies .

hive.optimize.distinct.rewrite
  • Default Value: true

  • Added In: Hive 1.2.0 with HIVE-10568

When applicable, this optimization rewrites distinct aggregates from a single-stage to multi-stage aggregation. This may not be optimal in all cases. Ideally, whether to trigger it or not should be a cost-based decision. Until Hive formalizes the cost model for this, this is config driven.

hive.optimize.point.lookup
  • Default Value: true

  • Added In: Hive 2.0.0 with HIVE-11461

Whether to transform OR clauses in Filter operators into IN clauses.

hive.optimize.point.lookup.min
  • Default Value: 31

  • Added In: Hive 2.0.0 with HIVE-11573

Minimum number of OR clauses needed to transform into IN clauses.

hive.allow.udf.load.on.demand
  • Default Value: false

  • Added In: Hive 2.1.0 with HIVE-13596

Whether enable loading UDFs from metastore on demand; this is mostly relevant for HS2 and was the default behavior before Hive 1.2.

hive.async.log.enabled
  • Default Value: true

  • Added In: Hive 2.1.0 with HIVE-13027

Whether to enable Log4j2's asynchronous logging. Asynchronous logging can give significant performance improvement as logging will be handled in a separate thread that uses the LMAX disruptor queue for buffering log messages.

Refer to https://logging.apache.org/log4j/2.x/manual/async.html  for benefits and drawbacks.

hive.msck.repair.batch.size
  • Default Value: 0
  • Added In: Hive 2.2.0 with HIVE-12077

To run the MSCK REPAIR TABLE command batch-wise. If there is a large number of untracked partitions, by configuring a value to the property it will execute in batches internally. The default value of the property is zero, which means it will execute all the partitions at once.

hive.exec.copyfile.maxnumfiles
  • Default Value: 1
  • Added In: Hive 2.3.0 with HIVE-14864

Maximum number of files Hive uses to do sequential HDFS copies between directories. Distributed copies (distcp) will be used instead for larger numbers of files so that copies can be done faster.

hive.exec.copyfile.maxsize
  • Default Value: 32 megabytes
  • Added In: Hive 1.1.0 with HIVE-8750

Maximum file size (in bytes) that Hive uses to do single HDFS copies between directories. Distributed copies (distcp) will be used instead for bigger files so that copies can be done faster.

hive.exec.stagingdir
  • Default Value: .hive-staging
  • Added in: Hive 1.1.0 with HIVE-8750

Directory name that will be created inside table locations in order to support HDFS encryption. This is replaces hive.exec.scratchdir for query results with the exception of read-only tables. In all cases hive.exec.scratchdir is still used for other temporary files, such as job plans.

hive.query.lifetime.hooks
  • Default Value: (empty)
  • Added In: Hive 2.3.0 with HIVE-14340

A comma separated list of hooks which implement QueryLifeTimeHook. These will be triggered before/after query compilation and before/after query execution, in the order specified. As of Hive 3.0.0 (HIVE-16363 ), this config can be used to specify implementations of QueryLifeTimeHookWithParseHooks. If they are specified then they will be invoked in the same places as QueryLifeTimeHooks and will be invoked during pre and post query parsing.

hive.remove.orderby.in.subquery
  • Default Value: true
  • Added In: Hive 3.0.0 with HIVE-6348

If set to true, order/sort by without limit in subqueries and views will be removed.

SerDes and I/O

SerDes

hive.script.serde
  • Default Value: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
  • Added In: Hive 0.4.0

The default SerDe for transmitting input data to and reading output data from the user scripts.

hive.script.recordreader
  • Default Value: org.apache.hadoop.hive.ql.exec.TextRecordReader
  • Added In: Hive 0.4.0

The default record reader for reading data from the user scripts.

hive.script.recordwriter
  • Default Value: org.apache.hadoop.hive.ql.exec.TextRecordWriter
  • Added In: Hive 0.5.0

The default record writer for writing data to the user scripts.

hive.default.serde
  • Default Value: org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe
  • Added in: Hive 0.14 with HIVE-5976

The default SerDe Hive will use for storage formats that do not specify a SerDe.  Storage formats that currently do not specify a SerDe include 'TextFile, RcFile'.  

See Registration of Native SerDes for more information for storage formats and SerDes.

hive.lazysimple.extended_boolean_literal
  • Default Value: false
  • Added in: Hive 0.14 with HIVE-3635

LazySimpleSerDe uses this property to determine if it treats 'T', 't', 'F', 'f', '1', and '0' as extended, legal boolean literals, in addition to 'TRUE' and 'FALSE'. The default is false, which means only 'TRUE' and 'FALSE' are treated as legal boolean literals.

I/O

hive.io .exception.handlers
  • Default Value: (empty)
  • Added In: Hive 0.8.1

A list of I/O exception handler class names. This is used to construct a list of exception handlers to handle exceptions thrown by record readers.

hive.input.format

The default input format. Set this to HiveInputFormat if you encounter problems with CombineHiveInputFormat.

Also see:

File Formats

hive.default.fileformat
  • Default Value: TextFile
  • Added In: Hive 0.2.0

Default file format for CREATE TABLE statement. Options are TextFile, SequenceFile, RCfile, ORC, and Parquet.

Users can explicitly say CREATE TABLE ... STORED AS TEXTFILE|SEQUENCEFILE|RCFILE|ORC|AVRO|INPUTFORMAT...OUTPUTFORMAT... to override. (RCFILE was added in Hive 0.6.0, ORC in 0.11.0, AVRO in 0.14.0, and Parquet in 2.3.0) See Row Format, Storage Format, and SerDe for details.

hive.default.fileformat.managed
  • Default Value: none
  • Added In: Hive 1.2.0 with HIVE-9915

Default file format for CREATE TABLE statement applied to managed tables only. External tables will be created with format specified by Configuration Properties#hive.default.fileformat . Options are none, TextFile, SequenceFile, RCfile, ORC, and Parquet (as of Hive 2.3.0). Leaving this null will result in using hive.default.fileformat for all native tables. For non-native tables the file format is determined by the storage handler, as shown below (see the StorageHandlers section for more information on managed/external and native/non-native terminology).


Native Non-Native
Managed hive.default.fileformat.managed (or fall back to hive.default.fileformat) Not covered by default file-formats
External hive.default.fileformat Not covered by default file-formats
hive.fileformat.check
  • Default Value: true
  • Added In: Hive 0.5.0

Whether to check file format or not when loading data files.

hive.query.result.fileformat
  • Default Value:
    • Hive 0.x, 1.x, and 2.0: TextFile
    • Hive 2.1 onward: SequenceFile
  • Added In: Hive 0.7.0 with HIVE-1598

File format to use for a query's intermediate results. Options are TextFile, SequenceFile, and RCfile. Default value is changed to SequenceFile since Hive 2.1.0 (HIVE-1608 ).

RCFile Format

hive.io .rcfile.record.interval
  • Default Value: 2147483647
  • Added In: Hive 0.4.0 with HIVE-352 ; added to HiveConf.java in Hive 0.14.0 with HIVE-7211
hive.io .rcfile.column.number.conf
  • Default Value: 0
  • Added In: Hive 0.4.0 with HIVE-352 ; added to HiveConf.java in Hive 0.14.0 with HIVE-7211
hive.io .rcfile.tolerate.corruptions
  • Default Value: false
  • Added In: Hive 0.4.0 with HIVE-352 ; added to HiveConf.java in Hive 0.14.0 with HIVE-7211
hive.io .rcfile.record.buffer.size
  • Default Value: 4194304
  • Added In: Hive 0.4.0 with HIVE-352 ; added to HiveConf.java in Hive 0.14.0 with HIVE-7211

ORC File Format

The ORC file format was introduced in Hive 0.11.0 . See ORC Files for details.

Besides the configuration properties listed in this section, some properties in other sections are also related to ORC:

hive.exec.orc.memory.pool
  • Default Value: 0.5
  • Added In: Hive 0.11.0 with HIVE-4248

Maximum fraction of heap that can be used by ORC file writers.

hive.exec.orc.write.format
  • Default Value: (empty)
  • Added In: Hive 0.12.0 with HIVE-4123 ; default changed from 0.11 to null with HIVE-5091 (also in Hive 0.12.0)

Define the version of the file to write. Possible values are 0.11 and 0.12. If this parameter is not defined, ORC will use the run length encoding (RLE) introduced in Hive 0.12. Any value other than 0.11 results in the 0.12 encoding.

Additional values may be introduced in the future (see HIVE-6002 ).

hive.exec.orc.base.delta.ratio
  • Default Value: 8
  • Added In: Hive 1.3.0 and 2.1.0 with HIVE-13563

Define the ratio of base writer and delta writer in terms of STRIPE_SIZE and BUFFER_SIZE.

hive.exec.orc.default.stripe.size
  • Default Value: 256*1024*1024 (268,435,456) in 0.13.0;
                             64*1024*1024 (67,108,864) in 0.14.0
  • Added In: Hive 0.13.0 with HIVE-5425 ; default changed in 0.14.0 with HIVE-7231 and HIVE-7490

Define the default ORC stripe size, in bytes.

hive.exec.orc.default.block.size
  • Default Value: 256*1024*1024 (268,435,456)
  • Added In: Hive 0.14.0 with HIVE-7231

Define the default file system block size for ORC files.

hive.exec.orc.dictionary.key.size.threshold
  • Default Value: 0.8
  • Added In: Hive 0.12.0 with HIVE-4324

If the number of keys in a dictionary is greater than this fraction of the total number of non-null rows, turn off dictionary encoding.  Use 1 to always use dictionary encoding.

hive.exec.orc.default.row.index.stride
  • Default Value: 10000
  • Added In: Hive 0.13.0 with HIVE-5728

Define the default ORC index stride in number of rows. (Stride is the number of rows an index entry represents.)

hive.exec.orc.default.buffer.size
  • Default Value: 256*1024 (262,144)
  • Added In: Hive 0.13.0 with HIVE-5728

Define the default ORC buffer size, in bytes.

hive.exec.orc.default.block.padding
  • Default Value: true
  • Added In: Hive 0.13.0 with HIVE-5728

Define the default block padding. Block padding was added in Hive 0.12.0 (HIVE-5091 , "ORC files should have an option to pad stripes to the HDFS block boundaries").

hive.exec.orc.block.padding.tolerance
  • Default Value: 0.05
  • Added In: Hive 0.14.0 with HIVE-7231

Define the tolerance for block padding as a decimal fraction of stripe size (for example, the default value 0.05 is 5% of the stripe size). For the defaults of 64Mb ORC stripe and 256Mb HDFS blocks, a maximum of 3.2Mb will be reserved for padding within the 256Mb block with the default hive.exec.orc.block.padding.tolerance. In that case, if the available size within the block is more than 3.2Mb, a new smaller stripe will be inserted to fit within that space. This will make sure that no stripe written will cross block boundaries and cause remote reads within a node local task.

hive.exec.orc.default.compress
  • Default Value: ZLIB
  • Added In: Hive 0.13.0 with HIVE-5728

Define the default compression codec for ORC file.

hive.exec.orc.encoding.strategy
  • Default Value: SPEED
  • Added In: Hive 0.14.0 with HIVE-7219

Define the encoding strategy to use while writing data. Changing this will only affect the light weight encoding for integers. This flag will not change the compression level of higher level compression codec (like ZLIB). Possible options are SPEED and COMPRESSION.

hive.orc.splits.include.file.footer

If turned on, splits generated by ORC will include metadata about the stripes in the file. This data is read remotely (from the client or HiveServer2 machine) and sent to all the tasks.

hive.orc.cache.stripe.details.size

Cache size for keeping meta information about ORC splits cached in the client.

hive.orc.cache.use.soft.references
  • Default Value: false
  • Added In: Hive 1.3.0, Hive 2.1.1, Hive 2.2.0 with HIVE-13985

By default, the cache that ORC input format uses to store the ORC file footer uses hard references for the cached object. Setting this to true can help avoid out-of-memory issues under memory pressure (in some cases) at the cost of slight unpredictability in overall query performance.

hive.io.sarg.cache.max.weight.mb
  • Default Value: 10
  • Added In: Hive 2.2.1, Hive 2.3.1, Hive 2.4.0, Hive 3.0.0 with HIVE-17669

The maximum weight allowed for the SearchArgument Cache, in megabytes. By default, the cache allows a max-weight of 10MB, after which entries will be evicted. Set to 0, to disable SearchArgument caching entirely.

hive.orc.compute.splits.num.threads

How many threads ORC should use to create splits in parallel.

hive.exec.orc.split.strategy
  • Default Value: HYBRID
  • Added In: Hive 1.2.0 with HIVE-10114

What strategy ORC should use to create splits for execution. The available options are "BI", "ETL" and "HYBRID".

The HYBRID mode reads the footers for all files if there are fewer files than expected mapper count, switching over to generating 1 split per file if the average file sizes are smaller than the default HDFS blocksize. ETL strategy always reads the ORC footers before generating splits, while the BI strategy generates per-file splits fast without reading any data from HDFS.

hive.exec.orc.skip.corrupt.data
  • Default Value: false
  • Added In: Hive 0.13.0 with HIVE-6382

If ORC reader encounters corrupt data, this value will be used to determine whether to skip the corrupt data or throw an exception. The default behavior is to throw an exception.

hive.exec.orc.zerocopy

Use zerocopy reads with ORC. (This requires Hadoop 2.3 or later.)

hive.merge.orcfile.stripe.level
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-7509

When Configuration Properties#hive.merge.mapfiles , Configuration Properties#hive.merge.mapredfiles or Configuration Properties#hive.merge.tezfiles is enabled while writing a table with ORC file format, enabling this configuration property will do stripe-level fast merge for small ORC files. Note that enabling this configuration property will not honor the padding tolerance configuration (Configuration Properties#hive.exec.orc.block.padding.tolerance ).

hive.orc.row.index.stride.dictionary.check
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-7832

If enabled dictionary check will happen after first row index stride (default 10000 rows) else dictionary check will happen before writing first stripe. In both cases, the decision to use dictionary or not will be retained thereafter.

hive.exec.orc.compression.strategy
  • Default Value: SPEED
  • Added In: Hive 0.14.0 with HIVE-7859

Define the compression strategy to use while writing data. This changes the compression level of higher level compression codec (like ZLIB).

Value can be SPEED or COMPRESSION.

Parquet

Parquet is supported by a plugin in Hive 0.10, 0.11, and 0.12 and natively in Hive 0.13 and later. See Parquet for details.

hive.parquet.timestamp.skip.conversion
  • Default Value: true
  • Added In: Hive 1.2.0 with HIVE-9482

Pre-3.1.2 Hive implementation of Parquet stores timestamps in UTC on-file, this flag allows skipping of the conversion on reading Parquet files created from other tools that may not have done so.

Avro

See AvroSerDe for details.

hive.avro.timestamp.skip.conversion
  • Default Value: false
  • Added In: Hive 3.1.2 with HIVE-21291

Some older Hive implementations (pre-3.1.2) wrote Avro timestamps in a UTC-normalized manner, while from version 3.1.0 until 3.1.2 Hive wrote time zone agnostic timestamps.
Setting this flag to true will treat legacy timestamps as time zone agnostic. Setting it to false will treat legacy timestamps as UTC-normalized.
This flag does not affect timestamps written starting with Hive 3.1.2, which are effectively time zone agnostic (see HIVE-21002  for details).
NOTE: This property will influence how HBase files using the AvroSerDe and timestamps in Kafka tables (in the payload/Avro file, this is not about Kafka timestamps) are deserialized – keep in mind that timestamps serialized using the AvroSerDe will be UTC-normalized during serialization. So keep this property false if using HBase or Kafka.

Vectorization

Hive added vectorized query execution in release 0.13.0 (HIVE-4160 , HIVE-5283 ). For more information see the design document Vectorized Query Execution.

hive.vectorized.execution.enabled
  • Default Value: false
  • Added In: Hive 0.13.0 with HIVE-5283

This flag should be set to true to enable vectorized mode of query execution. The default value is false.

hive.vectorized.execution.reduce.enabled
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-7405

This flag should be set to true to enable vectorized mode of the reduce-side of query execution. The default value is true.

hive.vectorized.execution.reduce.groupby.enabled
  • Default Value: true
  • Added In: Hive 0.14.0 with HIVE-8052

This flag should be set to true to enable vectorized mode of the reduce-side GROUP BY query execution. The default value is true.

hive.vectorized.execution.reducesink.new.enabled
  • Default Value: true
  • Added In: Hive 2.0.0 with HIVE-12290

This flag should be set to true to enable the new vectorization of queries using ReduceSink.

hive.vectorized.execution.mapjoin.native.enabled
  • Default Value: true
  • Added In: Hive 1.2.0 with HIVE-9824

This flag should be set to true to enable native (i.e. non-pass through) vectorization of queries using MapJoin.

hive.vectorized.execution.mapjoin.native.multikey.only.enabled
  • Default Value: false
  • Added In: Hive 1.2.0 with HIVE-9824

This flag should be set to true to restrict use of native vector map join hash tables to the MultiKey in queries using MapJoin.

hive.vectorized.execution.mapjoin.minmax.enabled
  • Default Value: false
  • Added In: Hive 1.2.0 with HIVE-9824

This flag should be set to true to enable vector map join hash tables to use max / max filtering for integer join queries using MapJoin.

hive.vectorized.execution.mapjoin.overflow.repeated.threshold
  • Default Value: -1
  • Added In: Hive 1.2.0 with HIVE-9824

The number of small table rows for a match in vector map join hash tables where we use the repeated field optimization in overflow vectorized row batch for join queries using MapJoin. A value of -1 means do use the join result optimization. Otherwise, threshold value can be 0 to maximum integer.

hive.vectorized.execution.mapjoin.native.fast.hashtable.enabled
  • Default Value: false
  • Added In: Hive 1.2.0 with HIVE-9824

This flag should be set to true to enable use of native fast vector map join hash tables in queries using MapJoin.

hive.vectorized.groupby.checkinterval
  • Default Value: 100000
  • Added In: Hive 0.13.0 with HIVE-5692

Number of entries added to the GROUP BY aggregation hash before a recomputation of average entry size is performed.

hive.vectorized.groupby.maxentries
  • Default Value: 1000000
  • Added In: Hive 0.13.0 with HIVE-5692

Maximum number of entries in the vector GROUP BY aggregation hashtables. Exceeding this will trigger a flush regardless of memory pressure condition.

hive.vectorized.use.vectorized.input.format
  • Default Value: true
  • Added In: Hive 2.1.0 with HIVE-12878

This flag should be set to true to allow Hive to take advantage of input formats that support vectorization. The default value is true.

hive.vectorized.use.vector.serde.deserialize
  • Default Value: false
  • Added In: Hive 2.1.0 with HIVE-12878

This flag should be set to true to enable vectorizing rows using vector deserialize. The default value is false.

hive.vectorized.use.row.serde.deserialize
  • Default Value: false
  • Added In: Hive 2.1.0 with HIVE-12878

This flag should be set to true to enable vectorizing using row deserialize. The default value is false.

hive.vectorized.input.format.excludes
  • Default Value: (empty)
  • Added in: Hive 2.4.0 with HIVE-17534

This flag should be used to provide a comma separated list of fully qualified classnames to exclude certain FileInputFormats from vectorized execution using the vectorized file inputformat. Note that vectorized execution could still occur for that input format based on whether Configuration Properties#hive.vectorized.use.vector.serde.deserialize or Configuration Properties#hive.vectorized.use.row.serde.deserialize is enabled or not. 

MetaStore

In addition to the Hive metastore properties listed in this section, some properties are listed in other sections:

hive.metastore.local
  • Default Value: true
  • Added In: Hive 0.8.1
  • Removed In: Hive 0.10 with HIVE-2585

Controls whether to connect to remote metastore server or open a new metastore server in Hive Client JVM. As of Hive 0.10 this is no longer used. Instead if hive.metastore.uris is set then remote mode is assumed otherwise local.

hive.metastore.uri.selection
  • Default Value: RANDOM
  • Added In: Hive 3.0.0

Determines the selection mechanism used by metastore client to connect to remote metastore. SEQUENTIAL implies that the first valid metastore from the URIs specified as part of hive.metastore.uris will be picked. RANDOM implies that the metastore will be picked randomly.

javax.jdo.option.ConnectionURL
  • Default Value: jdbc:derby:;databaseName=metastore_db;create=true
  • Added In: Hive 0.6.0

JDBC connect string for a JDBC metastore.

javax.jdo.option.ConnectionDriverName
  • Default Value: org.apache.derby.jdbc.EmbeddedDriver
  • Added In: Hive 0.8.1

Driver class name for a JDBC metastore.

javax.jdo.PersistenceManagerFactoryClass
  • Default Value: org.datanucleus.jdo.JDOPersistenceManagerFactory
  • Added In: Hive 0.8.1

Class implementing the JDO PersistenceManagerFactory.

javax.jdo.option.DetachAllOnCommit
  • Default Value: true
  • Added In: Hive 0.8.1

Detaches all objects from session so that they can be used after transaction is committed.

javax.jdo.option.NonTransactionalRead
  • Default Value: true
  • Added In: Hive 0.8.1

Reads outside of transactions.

javax.jdo.option.ConnectionUserName
  • Default Value: APP
  • Added In: Hive 0.8.1

Username to use against metastore database.

javax.jdo.option.ConnectionPassword
  • Default Value: mine
  • Added In: Hive 0.3.0

Password to use against metastore database.

For an alternative configuration, see Removing Hive Metastore Password from Hive Configuration.

javax.jdo.option.Multithreaded
  • Default Value: true
  • Added In: Hive 0.8.0

Set this to true if multiple threads access metastore through JDO concurrently.

datanucleus.connectionPoolingType
  • Default Value: DBCP in Hive 0.7 to 0.11; BoneCP in 0.12 to 2.3; HikariCP in 3.0 and later