Hive Configuration Properties

This document describes user configuration properties (sometimes called parameters, variables, or options) for Hive and notes some of the releases that 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.

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

Query and DDL Execution

hive.execution.engine

Chooses execution engine. Options are: mr (Map reduce, default) or tez (Tez execution, for Hadoop 2 only).

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

mapred.reduce.tasks

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

Size per reducer. The default is 1G, that is, if the input size is 10G then 10 reducers will be used.

hive.exec.reducers.max

Max number of reducers will be used. If the one specified in the configuration property mapred.reduce.tasks is negative, Hive will use this one as the max number of reducers when automatically determine number of reducers.

hive.exec.scratchdir

Scratch space for Hive jobs.

hive.default.fileformat

Default file format for CREATE TABLE statement. Options are TextFile, SequenceFile, RCfile, and ORC. Users can explicitly say CREATE TABLE ... STORED AS TEXTFILE|SEQUENCEFILE|RCFILE|ORC to override.

hive.fileformat.check

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

hive.query.result.fileformat

File format to use for a query's intermediate results. Options are TextFile, SequenceFile, and RCfile. Set to SequenceFile if any columns are string type and contain new-line characters (HIVE-1608, HIVE-3065).

hive.io.rcfile.record.interval
hive.io.rcfile.column.number.conf
hive.io.rcfile.tolerate.corruptions
hive.io.rcfile.record.buffer.size
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.compute.splits.num.threads

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

hive.exec.orc.zerocopy

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

hive.exec.orc.skip.corrupt.data

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.skip.corrupt.data

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.hadoop.supports.splittable.combineinputformat

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

hive.map.aggr

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

hive.groupby.skewindata

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

hive.groupby.mapaggr.checkinterval

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

hive.new.job.grouping.set.cardinality

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

For local mode, memory of the mappers/reducers.

hive.map.aggr.hash.force.flush.memory.threshold

The max 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

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

hive.map.aggr.hash.min.reduction

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

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

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.

hive.multigroupby.singlereducer

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

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

hive.optimize.index.filter

Whether to enable automatic use of indexes.

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

hive.optimize.ppd

Whether to enable predicate pushdown.

hive.optimize.ppd.storage

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

hive.ppd.recognizetransivity

Whether to transitively replicate predicate filters over equijoin conditions.

hive.join.emit.interval

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

hive.join.cache.size

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

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
hive.mapjoin.smalltable.filesize

The threshold 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 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

This number means how much memory the local task can take to hold the key/value into in-memory hash table when this map join followed by a group by; 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 the memory.

hive.mapjoin.check.memory.rows

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

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.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.optimize.skewjoin

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

hive.skewjoin.key

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

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

hive.skewjoin.mapjoin.min.split

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 hive.skewjoin.mapjoin.map.tasks to perform a fine grained control.

hive.optimize.skewjoin.compiletime

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 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 hive.optimize.skewjoin should be set to true. (Ideally, 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 hive.optimize.skewjoin will be a no-op.

hive.optimize.union.remove

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 hive.optimize.skewjoin.compiletime is set to true, since an extra union is inserted.

The merge is triggered if either of hive.merge.mapfiles or hive.merge.mapredfiles is set to true. If the user has set hive.merge.mapfiles to true and 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

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

The mode in which the Hive operations are being performed. In strict mode, some risky queries are not allowed to run.

hive.exec.script.maxerrsize

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.exec.script.allow.partial.consumption

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

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.exec.compress.output

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

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

Whether to execute jobs in parallel.

hive.exec.parallel.thread.number

How many jobs at most can be executed in parallel.

hive.exec.rowoffset

Whether to provide the row offset virtual column.

hive.task.progress

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 hive.exec.dynamic.partition set to true.

hive.counters.group.name

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

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

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

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

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

hive.merge.mapredfiles

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

hive.mergejob.maponly

Try to generate a map-only job for merging files if CombineHiveInputFormat is supported.

hive.merge.size.per.task

Size of merged files at the end of the job.

hive.merge.smallfiles.avgsize

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

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

hive.auto.convert.join

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

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

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.script.auto.progress

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.script.serde

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

hive.script.recordreader

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

hive.script.recordwriter

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

hive.input.format

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

hive.udtf.auto.progress

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

Whether speculative execution for reducers should be turned on.

hive.exec.counters.pull.interval

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

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

hive.enforce.sorting

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

hive.optimize.reducededuplication

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.exec.dynamic.partition

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

hive.exec.dynamic.partition.mode

In strict mode, the user must specify at least one static partition in case the user accidentally overwrites all partitions.

hive.exec.max.dynamic.partitions

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

hive.exec.max.dynamic.partitions.pernode

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

hive.exec.max.created.files

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

hive.exec.default.partition.name

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

The SerDe used by FetchTask to serialize the fetch output.

hive.exec.mode.local.auto

Let Hive determine whether to run in local mode automatically.

hive.exec.drop.ignorenonexistent

Do not report an error if DROP TABLE/VIEW specifies a non-existent table/view.

hive.exec.show.job.failure.debug.info

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

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

hive.table.parameters.default

Default property values for newly created tables.

hive.variable.substitute

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

hive.error.on.empty.partition

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

hive.exim.uri.scheme.whitelist

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

hive.limit.row.max.size

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

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

hive.limit.optimize.enable

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

hive.limit.optimize.fetch.max

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

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

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

hive.io.exception.handlers

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.autogen.columnalias.prefix.label

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

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

hive.exec.perf.logger

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 cleanup the Hive scratch directory while starting the Hive server.

hive.output.file.extension

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

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

hive.resultset.use.unique.column.names

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

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.display.partition.cols.separately

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.

MetaStore

hive.metastore.local

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.

javax.jdo.option.ConnectionURL

JDBC connect string for a JDBC metastore.

javax.jdo.option.ConnectionDriverName

Driver class name for a JDBC metastore.

javax.jdo.PersistenceManagerFactoryClass

Class implementing the JDO PersistenceManagerFactory.

javax.jdo.option.DetachAllOnCommit

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

javax.jdo.option.NonTransactionalRead

Reads outside of transactions.

javax.jdo.option.ConnectionUserName

Username to use against metastore database.

javax.jdo.option.ConnectionPassword

Password to use against metastore database.

javax.jdo.option.Multithreaded

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

datanucleus.connectionPoolingType

Uses a BoneCP connection pool for JDBC metastore in release 0.12 and later (HIVE-4807), or a DBCP connection pool in releases 0.7 to 0.11.

datanucleus.validateTables

Validates existing schema against code. Turn this on if you want to verify existing schema

datanucleus.validateColumns

Validates existing schema against code. Turn this on if you want to verify existing schema.

datanucleus.validateConstraints

Validates existing schema against code. Turn this on if you want to verify existing schema.

datanucleus.storeManagerType

Metadata store type.

datanucleus.autoCreateSchema

Creates necessary schema on a startup if one doesn't exist. Set this to false, after creating it once.

In Hive 0.12.0 and later releases, datanucleus.autoCreateSchema is disabled if hive.metastore.schema.verification is true.

datanucleus.autoStartMechanismMode

Throw exception if metadata tables are incorrect.

datanucleus.transactionIsolation

Default transaction isolation level for identity generation.

datanucleus.cache.level2

This parameter does nothing.
Warning note: For most installations, Hive should not enable the DataNucleus L2 cache, since this can cause correctness issues. Thus, some people set this parameter to false assuming that this disables the cache – unfortunately, it does not. To actually disable the cache, set datanucleus.cache.level2.type to "none".

datanucleus.cache.level2.type

NONE = disable the datanucleus level 2 cache, SOFT = soft reference based cache, WEAK = weak reference based cache.
Warning note: For most Hive installations, enabling the datanucleus cache can lead to correctness issues, and is dangerous. This should be left  as "none".

datanucleus.identifierFactory

Name of the identifier factory to use when generating table/column names etc. 'datanucleus' is used for backward compatibility.

datanucleus.plugin.pluginRegistryBundleCheck

Defines what happens when plugin bundles are found and are duplicated: EXCEPTION, LOG, or NONE.

hive.metastore.warehouse.dir

Location of default database for the warehouse.

hive.warehouse.subdir.inherit.perms

Set this to true if table directories should inherit the permissions of the warehouse or database directory instead of being created with permissions derived from dfs umask. (This configuration property replaced hive.files.umask.value before Hive 0.9.0 was released.)

hive.metastore.execute.setugi

In unsecure mode, true will cause the metastore to execute DFS operations using the client's reported user and group permissions. Note that this property must be set on both the client and server sides. Further note that it's best effort. If client sets it to true and server sets it to false, the client setting will be ignored.

hive.metastore.event.listeners

List of comma-separated listeners for metastore events.

hive.metastore.partition.inherit.table.properties

List of comma-separated keys occurring in table properties which will get inherited to newly created partitions. * implies all the keys will get inherited.

hive.metastore.end.function.listeners

List of comma-separated listeners for the end of metastore functions.

hive.metastore.event.expiry.duration

Duration after which events expire from events table (in seconds).

hive.metastore.event.clean.freq

Frequency at which timer task runs to purge expired events in metastore(in seconds).

hive.metastore.connect.retries

Number of retries while opening a connection to metastore.

hive.metastore.client.connect.retry.delay

Number of seconds for the client to wait between consecutive connection attempts.

hive.metastore.client.socket.timeout

MetaStore Client socket timeout in seconds.

hive.metastore.rawstore.impl

Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. This class is used to store and retrieval of raw metadata objects such as table, database.

hive.metastore.batch.retrieve.max

Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. The higher the number, the less the number of round trips is needed to the Hive metastore server, but it may also cause higher memory requirement at the client side.

hive.metastore.ds.connection.url.hook

Name of the hook to use for retriving the JDO connection URL. If empty, the value in javax.jdo.option.ConnectionURL is used.

hive.metastore.ds.retry.attempts

The number of times to retry a metastore call if there were a connection error.

hive.metastore.ds.retry.interval

The number of milliseconds between metastore retry attempts

hive.metastore.server.min.threads

Minimum number of worker threads in the Thrift server's pool.

hive.metastore.server.max.threads

Maximum number of worker threads in the Thrift server's pool.

hive.metastore.server.tcp.keepalive

Whether to enable TCP keepalive for the metastore server. Keepalive will prevent accumulation of half-open connections.

hive.metastore.sasl.enabled

If true, the metastore thrift interface will be secured with SASL. Clients must authenticate with Kerberos.

hive.metastore.kerberos.keytab.file

The path to the Kerberos Keytab file containing the metastore thrift server's service principal.

hive.metastore.kerberos.principal

The service principal for the metastore thrift server. The special string _HOST will be replaced automatically with the correct host name.

hive.metastore.cache.pinobjtypes

List of comma-separated metastore object types that should be pinned in the cache.

hive.metastore.authorization.storage.checks

Should the metastore do authorization checks against the underlying storage for operations like drop-partition (disallow the drop-partition if the user in question doesn't have permissions to delete the corresponding directory on the storage).

hive.metastore.schema.verification

Enforce metastore schema version consistency.
True: Verify that version information stored in metastore matches with one from Hive jars. Also disable automatic schema migration attempt (see datanucleus.autoCreateSchema). Users are required to manually migrate schema after Hive upgrade which ensures proper metastore schema migration.
False: Warn if the version information stored in metastore doesn't match with one from Hive jars.

For more information, see Metastore Schema Consistency and Upgrades.

hive.metastore.integral.jdo.pushdown

Allow JDO query pushdown for integral partition columns in metastore. Off by default. This improves metastore performance for integral columns, especially if there's a large number of partitions. However, it doesn't work correctly with integral values that are not normalized (for example, if they have leading zeroes like 0012). If metastore direct SQL is enabled and works (hive.metastore.try.direct.sql), this optimization is also irrelevant.

hive.metastore.try.direct.sql

Whether the Hive metastore should try to use direct SQL queries instead of the DataNucleus for certain read paths. This can improve metastore performance when fetching many partitions or column statistics by orders of magnitude; however, it is not guaranteed to work on all RDBMS-es and all versions. In case of SQL failures, the metastore will fall back to the DataNucleus, so it's safe even if SQL doesn't work for all queries on your datastore. If all SQL queries fail (for example, your metastore is backed by MongoDB), you might want to disable this to save the try-and-fall-back cost.

hive.metastore.try.direct.sql.ddl

Same as hive.metastore.try.direct.sql, for read statements within a transaction that modifies metastore data. Due to non-standard behavior in Postgres, if a direct SQL select query has incorrect syntax or something similar inside a transaction, the entire transaction will fail and fall-back to DataNucleus will not be possible. You should disable the usage of direct SQL inside transactions if that happens in your case.

HiveServer2

HiveServer2 was added in Hive 0.11.0 with HIVE-2935.  For more information see Setting Up HiveServer2 and HiveServer2 Clients.

hive.server2.thrift.port

Port number of HiveServer2 Thrift interface. Can be overridden by setting $HIVE_SERVER2_THRIFT_PORT.

hive.server2.thrift.bind.host

Bind host on which to run the HiveServer2 Thrift interface. Can be overridden by setting $HIVE_SERVER2_THRIFT_BIND_HOST.

hive.server2.thrift.min.worker.threads

Minimum number of Thrift worker threads.

hive.server2.thrift.max.worker.threads

Maximum number of Thrift worker threads.

hive.server2.authentication

Client authentication types.

NONE: no authentication check
LDAP: LDAP/AD based authentication
KERBEROS: Kerberos/GSSAPI authentication
CUSTOM: Custom authentication provider (use with property hive.server2.custom.authentication.class)
PAM: Pluggable authentication module (added in Hive 0.13.0 with HIVE-6466)

hive.server2.authentication.kerberos.keytab

Kerberos keytab file for server principal.

hive.server2.authentication.kerberos.principal

Kerberos server principal.

hive.server2.custom.authentication.class

Custom authentication class. Used when property hive.server2.authentication is set to 'CUSTOM'. Provided class must be a proper implementation of the interface org.apache.hive.service.auth.PasswdAuthenticationProvider. HiveServer2 will call its Authenticate(user, passed) method to authenticate requests. The implementation may optionally extend Hadoop's org.apache.hadoop.conf.Configured class to grab Hive's Configuration object.

hive.server2.enable.doAs

Setting this property to true will have HiveServer2 execute Hive operations as the user making the calls to it.

hive.server2.authentication.ldap.url

LDAP connection URL.

hive.server2.authentication.ldap.baseDN

LDAP base DN (distinguished name).

hive.server2.authentication.ldap.Domain

LDAP domain.

hive.server2.transport.mode

Server transport mode. Value can be "binary" or "http".

hive.server2.thrift.http.port

Port number when in HTTP mode.

hive.server2.thrift.http.path

Path component of URL endpoint when in HTTP mode.

hive.server2.thrift.http.min.worker.threads

Minimum number of worker threads when in HTTP mode.

hive.server2.thrift.http.max.worker.threads

Maximum number of worker threads when in HTTP mode.

hive.server2.thrift.sasl.qop

Sasl QOP value; set it to one of the following values to enable higher levels of protection for HiveServer2 communication with clients.

"auth" – authentication only (default)
"auth-int" – authentication plus integrity protection
"auth-conf" – authentication plus integrity and confidentiality protection

Note that hadoop.rpc.protection being set to a higher level than HiveServer2 does not make sense in most situations. HiveServer2 ignores hadoop.rpc.protection in favor of hive.server2.thrift.sasl.qop.

This is applicable only if HiveServer2 is configured to use Kerberos authentication.

hive.server2.async.exec.threads

Number of threads in the async thread pool for HiveServer2.

hive.server2.async.exec.shutdown.timeout

Time (in seconds) for which HiveServer2 shutdown will wait for async threads to terminate.

hive.server2.table.type.mapping

This setting reflects how HiveServer2 will report the table types for JDBC and other client implementations that retrieve the available tables and supported table types.

HIVE: Exposes Hive's native table types like MANAGED_TABLE, EXTERNAL_TABLE, VIRTUAL_VIEW
CLASSIC: More generic types like TABLE and VIEW

hive.server2.session.hook

Session-level hook for HiveServer2.

hive.server2.max.start.attempts

The number of times HiveServer2 will attempt to start before exiting, sleeping 60 seconds between retries. The default of 30 will keep trying for 30 minutes.

hive.server2.async.exec.wait.queue.size

Size of the wait queue for async thread pool in HiveServer2. After hitting this limit, the async thread pool will reject new requests.

hive.server2.async.exec.keepalive.time

Time (in seconds) that an idle HiveServer2 async thread (from the thread pool) will wait for a new task to arrive before terminating.

hive.server2.long.polling.timeout

Time in milliseconds that HiveServer2 will wait, before responding to asynchronous calls that use long polling.

hive.server2.allow.user.substitution

Allow alternate user to be specified as part of HiveServer2 open connection request.

hive.server2.authentication.spnego.keytab

Keytab file for SPNEGO principal, optional. A typical value would look like /etc/security/keytabs/spnego.service.keytab. This keytab would be used by HiveServer2 when Kerberos security is enabled and HTTP transport mode is used. This needs to be set only if SPNEGO is to be used in authentication.

SPNEGO authentication would be honored only if valid hive.server2.authentication.spnego.principal and hive.server2.authentication.spnego.keytab are specified.

hive.server2.authentication.spnego.principal

SPNEGO service principal, optional. A typical value would look like HTTP/_HOST@EXAMPLE.COM. The SPNEGO service principal would be used by HiveServer2 when Kerberos security is enabled and HTTP transport mode is used. This needs to be set only if SPNEGO is to be used in authentication.

hive.server2.authentication.pam.services

List of the underlying PAM services that should be used when hive.server2.authentication type is PAM. A file with the same name must exist in /etc/pam.d.

hive.server2.use.SSL

Set this to true for using SSL encryption in HiveServer2.

hive.server2.keystore.path

SSL certificate keystore location.

hive.server2.keystore.password

SSL certificate keystore password.

hive.server2.tez.default.queues

A list of comma separated values corresponding to YARN queues of the same name. When HiveServer2 is launched in Tez mode, this configuration needs to be set for multiple Tez sessions to run in parallel on the cluster.

hive.server2.tez.sessions.per.default.queue

A positive integer that determines the number of Tez sessions that should be launched on each of the queues specified by hive.server2.tez.default.queues. Determines the parallelism on each queue.

hive.server2.tez.initialize.default.sessions

This flag is used in HiveServer 2 to enable a user to use HiveServer 2 without turning on Tez for HiveServer 2. The user could potentially want to run queries over Tez without the pool of sessions.

Tez

Apache Tez was added in Hive 0.13.0 (HIVE-4660 and HIVE-6098).  For information see the design document Hive on Tez.

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

hive.jar.directory

This is the location that Hive in Tez mode will look for to find a site-wide installed Hive instance.  See hive.user.install.directory for the default behavior.

hive.user.install.directory

If Hive (in Tez mode only) cannot find a usable Hive jar in hive.jar.directory, it will upload the Hive jar to <hive.user.install.directory>/<user_name> and use it to run queries.

hive.compute.splits.in.am

Whether to generate the splits locally or in the ApplicationMaster (Tez only).

hive.rpc.query.plan

Whether to send the query plan via local resource or RPC.

hive.prewarm.enabled

Enables container prewarm for Tez (Hadoop 2 only).

hive.prewarm.numcontainers

Controls the number of containers to prewarm for Tez (Hadoop 2 only).

hive.hive.merge.tezfiles

Merge small files at the end of a Tez DAG.

hive.tez.input.format

The default input format for Tez. Tez groups splits in the AM (ApplicationMaster).

hive.tez.container.size

By default Tez will spawn containers of the size of a mapper. This can be used to overwrite the default.

hive.tez.java.opts

By default Tez will use the Java options from map tasks. This can be used to overwrite the default.

hive.convert.join.bucket.mapjoin.tez

Whether joins can be automatically converted to bucket map joins in Hive when Tez is used as the execution engine (hive.execution.engine is set to "tez").

hive.tez.log.level

The log level to use for tasks executing as part of the DAG. Used only if hive.tez.java.opts is used to configure Java options.

hive.localize.resource.wait.interval

Time in milliseconds to wait for another thread to localize the same resource for Hive-Tez.

hive.localize.resource.num.wait.attempts

The number of attempts waiting for localizing a resource in Hive-Tez.

Indexing

Indexing was added in Hive 0.7.0 with HIVE-417, and bitmap indexing was added in Hive 0.8.0 with HIVE-1803.  For more information see Indexing.

hive.index.compact.file.ignore.hdfs

When true the HDFS location stored in the index file will be ignored at runtime. If the data got moved or the name of the cluster got changed, the index data should still be usable.

hive.optimize.index.filter

Whether to enable automatic use of indexes.

hive.optimize.index.filter.compact.minsize

Minimum size (in bytes) of the inputs on which a compact index is automatically used.

hive.optimize.index.filter.compact.maxsize

Maximum size (in bytes) of the inputs on which a compact index is automatically used. A negative number is equivalent to infinity.

hive.index.compact.query.max.size

The maximum number of bytes that a query using the compact index can read. Negative value is equivalent to infinity.

hive.index.compact.query.max.entries

The maximum number of index entries to read during a query that uses the compact index. Negative value is equivalent to infinity.

hive.exec.concatenate.check.index

If this sets to true, Hive will throw error when doing ALTER TABLE tbl_name [partSpec] CONCATENATE on a table/partition that has indexes on it. The reason the user want to set this to true is because it can help user to avoid handling all index drop, recreation, rebuild work. This is very helpful for tables with thousands of partitions.

hive.optimize.index.groupby
hive.index.compact.binary.search

Whether or not to use a binary search to find the entries in an index table that match the filter, where possible.

Statistics

See Statistics in Hive for information about how to collect and use Hive table statistics. 

hive.stats.dbclass

Hive 0.7 to 0.12:  The default database that stores temporary Hive statistics.  Other options are jdbc:mysql and hbase as defined in StatsSetupConst.java.

Hive 0.13 and later:  The storage that stores temporary Hive statistics. In FS based statistics collection, each task writes statistics it has collected in a file on the filesystem, which will be aggregated after the job has finished. Supported values are fs (filesystem), jdbc(:.*), hbase, counter and custom (HIVE-6500).

hive.stats.autogather

A flag to gather statistics automatically during the INSERT OVERWRITE command.

hive.stats.jdbcdriver

The JDBC driver for the database that stores temporary Hive statistics.

hive.stats.dbconnectionstring

The default connection string for the database that stores temporary Hive statistics.

hive.stats.default.publisher

The Java class (implementing the StatsPublisher interface) that is used by default if hive.stats.dbclass is not JDBC or HBase (Hive 0.12.0 and earlier), or if hive.stats.dbclass is a custom type (Hive 0.13.0 and later:  HIVE-4632).

hive.stats.default.aggregator

The Java class (implementing the StatsAggregator interface) that is used by default if hive.stats.dbclass is not JDBC or HBase (Hive 0.12.0 and earlier), or if hive.stats.dbclass is a custom type (Hive 0.13.0 and later:  HIVE-4632).

hive.stats.jdbc.timeout

Timeout value (number of seconds) used by JDBC connection and statements.

hive.stats.atomic

If this is set to true then the metastore statistics will be updated only if all types of statistics (number of rows, number of files, number of bytes, etc.) are available. Otherwise metastore statistics are updated in a best effort fashion with whatever are available.

hive.stats.retries.max

Maximum number of retries when stats publisher/aggregator got an exception updating intermediate database. Default is no tries on failures.

hive.stats.retries.wait

The base waiting window (in milliseconds) before the next retry. The actual wait time is calculated by baseWindow * failues + baseWindow * (failure + 1) * (random number between 0.0,1.0).

hive.stats.collect.rawdatasize

If true, the raw data size is collected when analyzing tables.

hive.client.stats.publishers

Comma-separated list of statistics publishers to be invoked on counters on each job. A client stats publisher is specified as the name of a Java class which implements the org.apache.hadoop.hive.ql.stats.ClientStatsPublisher interface.

hive.client.stats.counters

Subset of counters that should be of interest for hive.client.stats.publishers (when one wants to limit their publishing). Non-display names should be used.

hive.stats.reliable

Whether queries will fail because statistics cannot be collected completely accurately. If this is set to true, reading/writing from/into a partition or unpartitioned table may fail because the statistics could not be computed accurately. If it is set to false, the operation will succeed.

In Hive 0.13.0 and later, if hive.stats.reliable is false and statistics could not be computed correctly, the operation can still succeed and update the statistics but it sets a partition property "areStatsAccurate" to false. If the application needs accurate statistics, they can then be obtained in the background.

hive.stats.ndv.error

Standard error allowed for NDV estimates, expressed in percentage. This provides a tradeoff between accuracy and compute cost. A lower value for the error indicates higher accuracy and a higher compute cost. (NDV means number of distinct values.)

hive.stats.collect.tablekeys

Whether join and group by keys on tables are derived and maintained in the QueryPlan. This is useful to identify how tables are accessed and to determine if they should be bucketed.

hive.stats.collect.scancols

Whether column accesses are tracked in the QueryPlan. This is useful to identify how tables are accessed and to determine if there are wasted columns that can be trimmed.

hive.stats.key.prefix.max.length

Determines if, when the prefix of the key used for intermediate statistics collection exceeds a certain length, a hash of the key is used instead. If the value < 0 then hashing is never used, if the value >= 0 then hashing is used only when the key prefixes' length exceeds that value. The key prefix is defined as everything preceding the task ID in the key. For counter type statistics, it's maxed by mapreduce.job.counters.group.name.max, which is by default 128.

hive.stats.key.prefix.reserve.length

Reserved length for postfix of statistics key. Currently only meaningful for counter type statistics which should keep the length of the full statistics key smaller than the maximum length configured by hive.stats.key.prefix.max.length. For counter type statistics, it should be bigger than the length of LB spec if exists.

hive.stats.max.variable.length

To estimate the size of data flowing through operators in Hive/Tez (for reducer estimation etc.), average row size is multiplied with the total number of rows coming out of each operator. Average row size is computed from average column size of all columns in the row. In the absence of column statistics, for variable length columns (like string, bytes, etc.) this value will be used. For fixed length columns their corresponding Java equivalent sizes are used (float – 4 bytes, double – 8 bytes, etc.).

hive.stats.list.num.entries

To estimate the size of data flowing through operators in Hive/Tez (for reducer estimation etc.), average row size is multiplied with the total number of rows coming out of each operator. Average row size is computed from average column size of all columns in the row. In the absence of column statistics and for variable length complex columns like list, the average number of entries/values can be specified using this configuration property.

hive.stats.map.num.entries

To estimate the size of data flowing through operators in Hive/Tez (for reducer estimation etc.), average row size is multiplied with the total number of rows coming out of each operator. Average row size is computed from average column size of all columns in the row. In the absence of column statistics and for variable length complex columns like map, the average number of entries/values can be specified using this configuration property.

hive.stats.map.parallelism

The Hive/Tez optimizer estimates the data size flowing through each of the operators. For the GROUPBY operator, to accurately compute the data size map-side parallelism needs to be known. By default, this value is set to 1 since the optimizer is not aware of the number of mappers during compile-time. This Hive configuration property can be used to specify the number of mappers for data size computation of the GROUPBY operator.

hive.stats.fetch.partition.stats

Annotation of the operator tree with statistics information requires partition level basic statistics like number of rows, data size and file size. Partition statistics are fetched from the metastore. Fetching partition statistics for each needed partition can be expensive when the number of partitions is high. This flag can be used to disable fetching of partition statistics from the metastore. When this flag is disabled, Hive will make calls to the filesystem to get file sizes and will estimate the number of rows from the row schema.

hive.stats.fetch.column.stats

Annotation of the operator tree with statistics information requires column statistics. Column statistics are fetched from the metastore. Fetching column statistics for each needed column can be expensive when the number of columns is high. This flag can be used to disable fetching of column statistics from the metastore.

hive.stats.join.factor

The Hive/Tez optimizer estimates the data size flowing through each of the operators. The JOIN operator uses column statistics to estimate the number of rows flowing out of it and hence the data size. In the absence of column statistics, this factor determines the amount of rows flowing out of the JOIN operator.

hive.stats.deserialization.factor

The Hive/Tez optimizer estimates the data size flowing through each of the operators. In the absence of basic statistics like number of rows and data size, file size is used to estimate the number of rows and data size. Since files in tables/partitions are serialized (and optionally compressed) the estimates of number of rows and data size cannot be reliably determined. This factor is multiplied with the file size to account for serialization and compression.

hive.stats.avg.row.size

In the absence of table/partition statistics, average row size will be used to estimate the number of rows/data size.

hive.compute.query.using.stats

When set to true Hive will answer a few queries like min, max, and count(1) purely using statistics stored in the metastore. For basic statistics collection, set the configuration property hive.stats.autogather to true. For more advanced statistics collection, run ANALYZE TABLE queries.

Authentication/Authorization

Restricted List and Whitelist

hive.conf.restricted.list

Comma separated list of configuration properties which are immutable at runtime. For example, if hive.security.authorization.enabled is set to true, it should be included in this list to prevent a client from changing it to false at runtime.

hive.security.command.whitelist

Comma separated list of non-SQL Hive commands that users are authorized to execute. This can be used to restrict the set of authorized commands. The currently supported command list is "set,reset,dfs,add,delete,compile" and by default all these commands are authorized. To restrict any of these commands, set hive.security.command.whitelist to a value that does not have the command in it.

Hive Client Security

hive.security.authorization.enabled

Enable or disable the Hive client authorization.

hive.security.authorization.manager

The Hive client authorization manager class name. The user defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveAuthorizationProvider.

hive.security.authenticator.manager

Hive client authenticator manager class name. The user-defined authenticator should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.

hive.security.authorization.createtable.user.grants

The privileges automatically granted to some users whenever a table gets created. An example like "userX,userY:select;userZ:create" will grant select privilege to userX and userY, and grant create privilege to userZ whenever a new table created.

hive.security.authorization.createtable.group.grants

The privileges automatically granted to some groups whenever a table gets created. An example like "groupX,groupY:select;groupZ:create" will grant select privilege to groupX and groupY, and grant create privilege to groupZ whenever a new table created.

hive.security.authorization.createtable.role.grants

The privileges automatically granted to some roles whenever a table gets created. An example like "roleX,roleY:select;roleZ:create" will grant select privilege to roleX and roleY, and grant create privilege to roleZ whenever a new table created.

hive.security.authorization.createtable.owner.grants

The privileges automatically granted to the owner whenever a table gets created. An example like "select,drop" will grant select and drop privilege to the owner of the table.

Hive Metastore Security

Metastore-side security was added in Hive 0.10.0 (HIVE-3705).  For more information, see Metastore Server Security in the Authorization document.

hive.metastore.pre.event.listeners

The pre-event listener classes to be loaded on the metastore side to run code whenever databases, tables, and partitions are created, altered, or dropped. Set this configuration property to org.apache.hadoop.hive.ql.security.authorization.AuthorizationPreEventListener in hive-site.xml to turn on Hive metastore-side security.

hive.security.metastore.authorization.manager

The authorization manager class name to be used in the metastore for authorization. The user-defined authorization class should implement interface org.apache.hadoop.hive.ql.security.authorization.HiveMetastoreAuthorizationProvider. The DefaultHiveMetastoreAuthorizationProvider implements the standard Hive grant/revoke model. A storage-based authorization implementation is also provided to use as the value of this configuration property:

which uses HDFS permissions to provide authorization instead of using Hive-style grant-based authorization.

hive.security.metastore.authenticator.manager

The authenticator manager class name to be used in the metastore for authentication. The user-defined authenticator class should implement interface org.apache.hadoop.hive.ql.security.HiveAuthenticationProvider.

SQL Standard Based Authorization

Hive 0.13.0 introduces fine-grained authorization based on the SQL standard authorization model. This is still a work in progress – see HIVE-5837 for the functional specification and list of subtasks.

hive.users.in.admin.role

A comma separated list of users which will be added to the ADMIN role when the metastore starts up. More users can still be added later on.

Archiving

fs.har.impl

The implementation for accessing Hadoop Archives. Note that this won't be applicable to Hadoop versions less than 0.20.

hive.archive.enabled

Whether archiving operations are permitted.

hive.archive.har.parentdir.settable

In new Hadoop versions, the parent directory must be set while creating a HAR. Because this functionality is hard to detect with just version numbers, this configuration variable needs to be set manually.

Locking

See Hive Concurrency Model for general information about locking.

hive.support.concurrency

Whether Hive supports concurrency or not. A Zookeeper instance must be up and running for the default Hive lock manager to support read-write locks.

hive.lock.mapred.only.operation

This configuration property is to control whether or not only do lock on queries that need to execute at least one mapred job.

hive.lock.numretries

The number of times you want to try to get all the locks.

hive.unlock.numretries

The number of times you want to retry to do one unlock.

hive.lock.sleep.between.retries

The sleep time (in seconds) between various retries.

hive.zookeeper.quorum

The list of Zookeeper servers to talk to. This is only needed for read/write locks.

hive.zookeeper.client.port

The port of Zookeeper servers to talk to. This is only needed for read/write locks.

hive.zookeeper.session.timeout

Zookeeper client's session timeout. The client is disconnected, and as a result, all locks released, if a heartbeat is not sent in the timeout.

hive.zookeeper.namespace

The parent node under which all Zookeeper nodes are created.

hive.zookeeper.clean.extra.nodes

Clean extra nodes at the end of the session.

Clustering

hive.cluster.delegation.token.store.class

The delegation token store implementation. Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster.

hive.cluster.delegation.token.store.zookeeper.connectString

The ZooKeeper token store connect string.

hive.cluster.delegation.token.store.zookeeper.znode

The root path for token store data.

hive.cluster.delegation.token.store.zookeeper.acl

ACL for token store entries. List comma separated all server principals for the cluster.

Regions

hive.use.input.primary.region

When creating a table from an input table, create the table in the input table's primary region.

hive.default.region.name

The default region name.

hive.region.properties

The default filesystem and jobtracker for a region.

Command Line Interface

hive.cli.print.header

Whether to print the names of the columns in query output.

hive.cli.print.current.db

Whether to include the current database in the Hive prompt.

HBase StorageHandler

hive.hbase.wal.enabled

Whether writes to HBase should be forced to the write-ahead log. Disabling this improves HBase write performance at the risk of lost writes in case of a crash.

hive.hbase.generatehfiles

True when HBaseStorageHandler should generate hfiles instead of operate against the online table.

Hive Web Interface (HWI)

hive.hwi.war.file

This sets the path to the HWI war file, relative to ${HIVE_HOME}.

hive.hwi.listen.host

This is the host address the Hive Web Interface will listen on.

hive.hwi.listen.port

This is the port the Hive Web Interface will listen on.

Test Properties

hive.test.mode

Whether Hive is running in test mode. If yes, it turns on sampling and prefixes the output tablename.

hive.test.mode.prefix

If Hive is running in test mode, prefixes the output table by this string.

hive.test.mode.samplefreq

If Hive is running in test mode and table is not bucketed, sampling frequency.

hive.test.mode.nosamplelist

If Hive is running in test mode, don't sample the above comma separated list of tables.

HCatalog Configuration Properties

Starting in Hive release 0.11.0, HCatalog is installed and configured with Hive. The HCatalog server is the same as the Hive metastore. See Hive Metastore Administration for metastore configuration properties. For Hive releases prior to 0.11.0, see the "Thrift Server Setup" section in the HCatalog 0.5.0 document Installation from Tarball for information about setting the Hive metastore configuration properties.

Jobs submitted to HCatalog can specify configuration properties that affect storage, error tolerance, and other kinds of behavior during the job.  See HCatalog Config Properties for details.

WebHCat Configuration Properties

For WebHCat configuration, see Configuration Variables in the WebHCat manual.