Return-Path: X-Original-To: apmail-hive-commits-archive@www.apache.org Delivered-To: apmail-hive-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id F152510D97 for ; Tue, 18 Feb 2014 02:19:15 +0000 (UTC) Received: (qmail 14508 invoked by uid 500); 18 Feb 2014 02:19:15 -0000 Delivered-To: apmail-hive-commits-archive@hive.apache.org Received: (qmail 14469 invoked by uid 500); 18 Feb 2014 02:19:14 -0000 Mailing-List: contact commits-help@hive.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hive-dev@hive.apache.org Delivered-To: mailing list commits@hive.apache.org Received: (qmail 14461 invoked by uid 99); 18 Feb 2014 02:19:14 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 18 Feb 2014 02:19:14 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 18 Feb 2014 02:19:01 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id B8E2723888D7; Tue, 18 Feb 2014 02:18:38 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1569164 [2/5] - in /hive/trunk: common/ common/src/java/org/apache/hadoop/hive/ant/ common/src/java/org/apache/hadoop/hive/conf/ common/src/java/org/apache/hive/common/util/ common/src/test/org/apache/hadoop/hive/conf/ conf/ itests/hive-un... Date: Tue, 18 Feb 2014 02:18:37 -0000 To: commits@hive.apache.org From: brock@apache.org X-Mailer: svnmailer-1.0.9 Message-Id: <20140218021838.B8E2723888D7@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1569164&r1=1569163&r2=1569164&view=diff ============================================================================== --- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original) +++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Tue Feb 18 02:18:36 2014 @@ -27,17 +27,16 @@ import java.net.URL; import java.util.ArrayList; import java.util.HashMap; import java.util.Iterator; -import java.util.LinkedHashSet; import java.util.List; import java.util.Map; import java.util.Map.Entry; import java.util.Properties; -import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.security.auth.login.LoginException; +import static org.apache.hadoop.hive.conf.Validator.*; import org.apache.commons.lang.StringUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; @@ -166,839 +165,1378 @@ public class HiveConf extends Configurat */ public static enum ConfVars { // QL execution stuff - SCRIPTWRAPPER("hive.exec.script.wrapper", null), - PLAN("hive.exec.plan", ""), - PLAN_SERIALIZATION("hive.plan.serialization.format","kryo"), - SCRATCHDIR("hive.exec.scratchdir", "/tmp/hive-" + System.getProperty("user.name")), - LOCALSCRATCHDIR("hive.exec.local.scratchdir", System.getProperty("java.io.tmpdir") + File.separator + System.getProperty("user.name")), - SCRATCHDIRPERMISSION("hive.scratch.dir.permission", "700"), - SUBMITVIACHILD("hive.exec.submitviachild", false), - SCRIPTERRORLIMIT("hive.exec.script.maxerrsize", 100000), - ALLOWPARTIALCONSUMP("hive.exec.script.allow.partial.consumption", false), - STREAMREPORTERPERFIX("stream.stderr.reporter.prefix", "reporter:"), - STREAMREPORTERENABLED("stream.stderr.reporter.enabled", true), - COMPRESSRESULT("hive.exec.compress.output", false), - COMPRESSINTERMEDIATE("hive.exec.compress.intermediate", false), - COMPRESSINTERMEDIATECODEC("hive.intermediate.compression.codec", ""), - COMPRESSINTERMEDIATETYPE("hive.intermediate.compression.type", ""), - BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (1000 * 1000 * 1000)), - MAXREDUCERS("hive.exec.reducers.max", 999), - PREEXECHOOKS("hive.exec.pre.hooks", ""), - POSTEXECHOOKS("hive.exec.post.hooks", ""), - ONFAILUREHOOKS("hive.exec.failure.hooks", ""), - CLIENTSTATSPUBLISHERS("hive.client.stats.publishers", ""), - EXECPARALLEL("hive.exec.parallel", false), // parallel query launching - EXECPARALLETHREADNUMBER("hive.exec.parallel.thread.number", 8), - HIVESPECULATIVEEXECREDUCERS("hive.mapred.reduce.tasks.speculative.execution", true), - HIVECOUNTERSPULLINTERVAL("hive.exec.counters.pull.interval", 1000L), - DYNAMICPARTITIONING("hive.exec.dynamic.partition", true), - DYNAMICPARTITIONINGMODE("hive.exec.dynamic.partition.mode", "strict"), - DYNAMICPARTITIONMAXPARTS("hive.exec.max.dynamic.partitions", 1000), - DYNAMICPARTITIONMAXPARTSPERNODE("hive.exec.max.dynamic.partitions.pernode", 100), - MAXCREATEDFILES("hive.exec.max.created.files", 100000L), + SCRIPTWRAPPER("hive.exec.script.wrapper", null, ""), + PLAN("hive.exec.plan", "", ""), + PLAN_SERIALIZATION("hive.plan.serialization.format", "kryo", + "Query plan format serialization between client and task nodes. \n" + + "Two supported values are : kryo and javaXML. Kryo is default."), + SCRATCHDIR("hive.exec.scratchdir", + "/tmp/hive-" + System.getProperty("user.name"), + "Scratch space for Hive jobs"), + LOCALSCRATCHDIR("hive.exec.local.scratchdir", + System.getProperty("java.io.tmpdir") + File.separator + System.getProperty("user.name"), + "Local scratch space for Hive jobs"), + SCRATCHDIRPERMISSION("hive.scratch.dir.permission", "700", ""), + SUBMITVIACHILD("hive.exec.submitviachild", false, ""), + SCRIPTERRORLIMIT("hive.exec.script.maxerrsize", 100000, + "Maximum number of bytes a script is allowed to emit to standard error (per map-reduce task). \n" + + "This prevents runaway scripts from filling logs partitions to capacity"), + ALLOWPARTIALCONSUMP("hive.exec.script.allow.partial.consumption", false, + "When enabled, this option allows a user script to exit successfully without consuming all the data from the standard input."), + STREAMREPORTERPERFIX("stream.stderr.reporter.prefix", "reporter:", + "Streaming jobs that log to standard error with this prefix can log counter or status information."), + STREAMREPORTERENABLED("stream.stderr.reporter.enabled", true, + "Enable consumption of status and counter messages for streaming jobs."), + COMPRESSRESULT("hive.exec.compress.output", false, + "This controls whether the final outputs of a query (to a local/HDFS file or a Hive table) is compressed. \n" + + "The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"), + COMPRESSINTERMEDIATE("hive.exec.compress.intermediate", false, + "This controls whether intermediate files produced by Hive between multiple map-reduce jobs are compressed. \n" + + "The compression codec and other options are determined from Hadoop config variables mapred.output.compress*"), + COMPRESSINTERMEDIATECODEC("hive.intermediate.compression.codec", "", ""), + COMPRESSINTERMEDIATETYPE("hive.intermediate.compression.type", "", ""), + BYTESPERREDUCER("hive.exec.reducers.bytes.per.reducer", (long) (1000 * 1000 * 1000), + "size per reducer.The default is 1G, i.e if the input size is 10G, it will use 10 reducers."), + MAXREDUCERS("hive.exec.reducers.max", 999, + "max number of reducers will be used. If the one specified in the configuration parameter mapred.reduce.tasks is\n" + + "negative, Hive will use this one as the max number of reducers when automatically determine number of reducers."), + PREEXECHOOKS("hive.exec.pre.hooks", "", + "Comma-separated list of pre-execution hooks to be invoked for each statement. \n" + + "A pre-execution hook is specified as the name of a Java class which implements the \n" + + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), + POSTEXECHOOKS("hive.exec.post.hooks", "", + "Comma-separated list of post-execution hooks to be invoked for each statement. \n" + + "A post-execution hook is specified as the name of a Java class which implements the \n" + + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), + ONFAILUREHOOKS("hive.exec.failure.hooks", "", + "Comma-separated list of on-failure hooks to be invoked for each statement. \n" + + "An on-failure hook is specified as the name of Java class which implements the \n" + + "org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext interface."), + CLIENTSTATSPUBLISHERS("hive.client.stats.publishers", "", + "Comma-separated list of statistics publishers to be invoked on counters on each job. \n" + + "A client stats publisher is specified as the name of a Java class which implements the \n" + + "org.apache.hadoop.hive.ql.stats.ClientStatsPublisher interface."), + EXECPARALLEL("hive.exec.parallel", false, "Whether to execute jobs in parallel"), + EXECPARALLETHREADNUMBER("hive.exec.parallel.thread.number", 8, + "How many jobs at most can be executed in parallel"), + HIVESPECULATIVEEXECREDUCERS("hive.mapred.reduce.tasks.speculative.execution", true, + "Whether speculative execution for reducers should be turned on. "), + HIVECOUNTERSPULLINTERVAL("hive.exec.counters.pull.interval", 1000L, + "The interval with which to poll the JobTracker for the counters the running job. \n" + + "The smaller it is the more load there will be on the jobtracker, the higher it is the less granular the caught will be."), + DYNAMICPARTITIONING("hive.exec.dynamic.partition", true, + "Whether or not to allow dynamic partitions in DML/DDL."), + DYNAMICPARTITIONINGMODE("hive.exec.dynamic.partition.mode", "strict", + "In strict mode, the user must specify at least one static partition in case the user accidentally overwrites all partitions."), + DYNAMICPARTITIONMAXPARTS("hive.exec.max.dynamic.partitions", 1000, + "Maximum number of dynamic partitions allowed to be created in total."), + DYNAMICPARTITIONMAXPARTSPERNODE("hive.exec.max.dynamic.partitions.pernode", 100, + "Maximum number of dynamic partitions allowed to be created in each mapper/reducer node."), + MAXCREATEDFILES("hive.exec.max.created.files", 100000L, + "Maximum number of HDFS files created by all mappers/reducers in a MapReduce job."), DOWNLOADED_RESOURCES_DIR("hive.downloaded.resources.dir", - System.getProperty("java.io.tmpdir") + File.separator + "${hive.session.id}_resources"), - DEFAULTPARTITIONNAME("hive.exec.default.partition.name", "__HIVE_DEFAULT_PARTITION__"), - DEFAULT_ZOOKEEPER_PARTITION_NAME("hive.lockmgr.zookeeper.default.partition.name", "__HIVE_DEFAULT_ZOOKEEPER_PARTITION__"), + System.getProperty("java.io.tmpdir") + File.separator + "${hive.session.id}_resources", + "The default partition name in case the dynamic partition column value is null/empty string or any other values that cannot be escaped. \n" + + "This value must not contain any special character used in HDFS URI (e.g., ':', '%', '/' etc). \n" + + "The user has to be aware that the dynamic partition value should not contain this value to avoid confusions."), + DEFAULTPARTITIONNAME("hive.exec.default.partition.name", "__HIVE_DEFAULT_PARTITION__", ""), + DEFAULT_ZOOKEEPER_PARTITION_NAME("hive.lockmgr.zookeeper.default.partition.name", "__HIVE_DEFAULT_ZOOKEEPER_PARTITION__", ""), // Whether to show a link to the most failed task + debugging tips - SHOW_JOB_FAIL_DEBUG_INFO("hive.exec.show.job.failure.debug.info", true), - JOB_DEBUG_CAPTURE_STACKTRACES("hive.exec.job.debug.capture.stacktraces", true), - JOB_DEBUG_TIMEOUT("hive.exec.job.debug.timeout", 30000), - TASKLOG_DEBUG_TIMEOUT("hive.exec.tasklog.debug.timeout", 20000), - OUTPUT_FILE_EXTENSION("hive.output.file.extension", null), - - HIVE_IN_TEST("hive.in.test", false), // internal usage only, true in test mode - - // should hive determine whether to run in local mode automatically ? - LOCALMODEAUTO("hive.exec.mode.local.auto", false), - // if yes: - // run in local mode only if input bytes is less than this. 128MB by default - LOCALMODEMAXBYTES("hive.exec.mode.local.auto.inputbytes.max", 134217728L), - // run in local mode only if number of tasks (for map and reduce each) is - // less than this - LOCALMODEMAXINPUTFILES("hive.exec.mode.local.auto.input.files.max", 4), - // if true, DROP TABLE/VIEW does not fail if table/view doesn't exist and IF EXISTS is - // not specified - DROPIGNORESNONEXISTENT("hive.exec.drop.ignorenonexistent", true), - // ignore the mapjoin hint - HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true), + SHOW_JOB_FAIL_DEBUG_INFO("hive.exec.show.job.failure.debug.info", true, + "If a job fails, whether to provide a link in the CLI to the task with the\n" + + "most failures, along with debugging hints if applicable."), + JOB_DEBUG_CAPTURE_STACKTRACES("hive.exec.job.debug.capture.stacktraces", true, + "Whether or not stack traces parsed from the task logs of a sampled failed task \n" + + "for each failed job should be stored in the SessionState"), + JOB_DEBUG_TIMEOUT("hive.exec.job.debug.timeout", 30000, ""), + TASKLOG_DEBUG_TIMEOUT("hive.exec.tasklog.debug.timeout", 20000, ""), + OUTPUT_FILE_EXTENSION("hive.output.file.extension", null, + "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_IN_TEST("hive.in.test", false, "internal usage only, true in test mode", true), + + LOCALMODEAUTO("hive.exec.mode.local.auto", false, + "Let Hive determine whether to run in local mode automatically"), + LOCALMODEMAXBYTES("hive.exec.mode.local.auto.inputbytes.max", 134217728L, + "When hive.exec.mode.local.auto is true, input bytes should less than this for local mode."), + LOCALMODEMAXINPUTFILES("hive.exec.mode.local.auto.input.files.max", 4, + "When hive.exec.mode.local.auto is true, the number of tasks should less than this for local mode."), - // Max number of lines of footer user can set for a table file. - HIVE_FILE_MAX_FOOTER("hive.file.max.footer", 100), + DROPIGNORESNONEXISTENT("hive.exec.drop.ignorenonexistent", true, + "Do not report an error if DROP TABLE/VIEW specifies a non-existent table/view"), + + HIVEIGNOREMAPJOINHINT("hive.ignore.mapjoin.hint", true, "Ignore the mapjoin hint"), + + HIVE_FILE_MAX_FOOTER("hive.file.max.footer", 100, + "maximum number of lines for footer user can define for a table file"), // Hadoop Configuration Properties // Properties with null values are ignored and exist only for the purpose of giving us // a symbolic name to reference in the Hive source code. Properties with non-null // values will override any values set in the underlying Hadoop configuration. - HADOOPBIN("hadoop.bin.path", findHadoopBinary()), - HIVE_FS_HAR_IMPL("fs.har.impl", "org.apache.hadoop.hive.shims.HiveHarFileSystem"), - HADOOPFS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPFS"), null), - HADOOPMAPFILENAME(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPFILENAME"), null), - HADOOPMAPREDINPUTDIR(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPREDINPUTDIR"), null), - HADOOPMAPREDINPUTDIRRECURSIVE(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPREDINPUTDIRRECURSIVE"), false), - MAPREDMAXSPLITSIZE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMAXSPLITSIZE"), 256000000L), - MAPREDMINSPLITSIZE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZE"), 1L), - MAPREDMINSPLITSIZEPERNODE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZEPERNODE"), 1L), - MAPREDMINSPLITSIZEPERRACK(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZEPERRACK"), 1L), + HADOOPBIN("hadoop.bin.path", findHadoopBinary(), ""), + HIVE_FS_HAR_IMPL("fs.har.impl", "org.apache.hadoop.hive.shims.HiveHarFileSystem", + "The implementation for accessing Hadoop Archives. Note that this won't be applicable to Hadoop versions less than 0.20"), + HADOOPFS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPFS"), null, "", true), + HADOOPMAPFILENAME(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPFILENAME"), null, "", true), + HADOOPMAPREDINPUTDIR(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPREDINPUTDIR"), null, "", true), + HADOOPMAPREDINPUTDIRRECURSIVE(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPMAPREDINPUTDIRRECURSIVE"), false, "", true), + MAPREDMAXSPLITSIZE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMAXSPLITSIZE"), 256000000L, "", true), + MAPREDMINSPLITSIZE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZE"), 1L, "", true), + MAPREDMINSPLITSIZEPERNODE(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZEPERNODE"), 1L, "", true), + MAPREDMINSPLITSIZEPERRACK(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDMINSPLITSIZEPERRACK"), 1L, "", true), // The number of reduce tasks per job. Hadoop sets this value to 1 by default // By setting this property to -1, Hive will automatically determine the correct // number of reducers. - HADOOPNUMREDUCERS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPNUMREDUCERS"), -1), - HADOOPJOBNAME(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPJOBNAME"), null), - HADOOPSPECULATIVEEXECREDUCERS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPSPECULATIVEEXECREDUCERS"), true), - MAPREDSETUPCLEANUPNEEDED(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDSETUPCLEANUPNEEDED"), false), - MAPREDTASKCLEANUPNEEDED(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDTASKCLEANUPNEEDED"), false), + HADOOPNUMREDUCERS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPNUMREDUCERS"), -1, "", true), + HADOOPJOBNAME(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPJOBNAME"), null, "", true), + HADOOPSPECULATIVEEXECREDUCERS(ShimLoader.getHadoopShims().getHadoopConfNames().get("HADOOPSPECULATIVEEXECREDUCERS"), true, "", true), + MAPREDSETUPCLEANUPNEEDED(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDSETUPCLEANUPNEEDED"), false, "", true), + MAPREDTASKCLEANUPNEEDED(ShimLoader.getHadoopShims().getHadoopConfNames().get("MAPREDTASKCLEANUPNEEDED"), false, "", true), // Metastore stuff. Be sure to update HiveConf.metaVars when you add // something here! - METASTOREDIRECTORY("hive.metastore.metadb.dir", ""), - METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse"), - METASTOREURIS("hive.metastore.uris", ""), - // Number of times to retry a connection to a Thrift metastore server - METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3), - // Number of times to retry a Thrift metastore call upon failure - METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1), - - // Number of seconds the client should wait between connection attempts - METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", 1), - // Socket timeout for the client connection (in seconds) - METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", 20), - METASTOREPWD("javax.jdo.option.ConnectionPassword", "mine"), - // Class name of JDO connection url hook - METASTORECONNECTURLHOOK("hive.metastore.ds.connection.url.hook", ""), - METASTOREMULTITHREADED("javax.jdo.option.Multithreaded", true), - // Name of the connection url in the configuration + METASTOREDIRECTORY("hive.metastore.metadb.dir", "", ""), + METASTOREWAREHOUSE("hive.metastore.warehouse.dir", "/user/hive/warehouse", + "location of default database for the warehouse"), + METASTOREURIS("hive.metastore.uris", "", + "Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore."), + + METASTORETHRIFTCONNECTIONRETRIES("hive.metastore.connect.retries", 3, + "Number of retries while opening a connection to metastore"), + METASTORETHRIFTFAILURERETRIES("hive.metastore.failure.retries", 1, + "Number of retries upon failure of Thrift metastore calls"), + + METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", 1, + "Number of seconds for the client to wait between consecutive connection attempts"), + METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", 20, + "MetaStore Client socket timeout in seconds"), + METASTOREPWD("javax.jdo.option.ConnectionPassword", "mine", + "password to use against metastore database"), + METASTORECONNECTURLHOOK("hive.metastore.ds.connection.url.hook", "", + "Name of the hook to use for retrieving the JDO connection URL. If empty, the value in javax.jdo.option.ConnectionURL is used"), + METASTOREMULTITHREADED("javax.jdo.option.Multithreaded", true, + "Set this to true if multiple threads access metastore through JDO concurrently."), METASTORECONNECTURLKEY("javax.jdo.option.ConnectionURL", - "jdbc:derby:;databaseName=metastore_db;create=true"), - // Whether to force reloading of the metastore configuration (including - // the connection URL, before the next metastore query that accesses the - // datastore. Once reloaded, this value is reset to false. Used for - // testing only. - METASTOREFORCERELOADCONF("hive.metastore.force.reload.conf", false), - // Number of attempts to retry connecting after there is a JDO datastore err - HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 1), - // Number of miliseconds to wait between attepting - HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", 1000), - // Whether to force reloading of the HMSHandler configuration (including - // the connection URL, before the next metastore query that accesses the - // datastore. Once reloaded, this value is reset to false. Used for - // testing only. - HMSHANDLERFORCERELOADCONF("hive.hmshandler.force.reload.conf", false), - METASTORESERVERMINTHREADS("hive.metastore.server.min.threads", 200), - METASTORESERVERMAXTHREADS("hive.metastore.server.max.threads", 100000), - METASTORE_TCP_KEEP_ALIVE("hive.metastore.server.tcp.keepalive", true), - // Intermediate dir suffixes used for archiving. Not important what they - // are, as long as collisions are avoided + "jdbc:derby:;databaseName=metastore_db;create=true", + "JDBC connect string for a JDBC metastore"), + + METASTOREFORCERELOADCONF("hive.metastore.force.reload.conf", false, + "Whether to force reloading of the metastore configuration (including\n" + + "the connection URL, before the next metastore query that accesses the\n" + + "datastore. Once reloaded, this value is reset to false. Used for\n" + + "testing only.\n"), + HMSHANDLERATTEMPTS("hive.hmshandler.retry.attempts", 1, + "The number of times to retry a HMSHandler call if there were a connection error"), + HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", 1000, + "The number of milliseconds between HMSHandler retry attempts"), + HMSHANDLERFORCERELOADCONF("hive.hmshandler.force.reload.conf", false, + "Whether to force reloading of the HMSHandler configuration (including\n" + + "the connection URL, before the next metastore query that accesses the\n" + + "datastore. Once reloaded, this value is reset to false. Used for\n" + + "testing only.\n"), + METASTORESERVERMINTHREADS("hive.metastore.server.min.threads", 200, + "Minimum number of worker threads in the Thrift server's pool."), + METASTORESERVERMAXTHREADS("hive.metastore.server.max.threads", 100000, + "Maximum number of worker threads in the Thrift server's pool."), + METASTORE_TCP_KEEP_ALIVE("hive.metastore.server.tcp.keepalive", true, + "Whether to enable TCP keepalive for the metastore server. Keepalive will prevent accumulation of half-open connections."), + METASTORE_INT_ORIGINAL("hive.metastore.archive.intermediate.original", - "_INTERMEDIATE_ORIGINAL"), + "_INTERMEDIATE_ORIGINAL", + "Intermediate dir suffixes used for archiving. Not important what they\n" + + "are, as long as collisions are avoided\n"), METASTORE_INT_ARCHIVED("hive.metastore.archive.intermediate.archived", - "_INTERMEDIATE_ARCHIVED"), + "_INTERMEDIATE_ARCHIVED", ""), METASTORE_INT_EXTRACTED("hive.metastore.archive.intermediate.extracted", - "_INTERMEDIATE_EXTRACTED"), - METASTORE_KERBEROS_KEYTAB_FILE("hive.metastore.kerberos.keytab.file", ""), + "_INTERMEDIATE_EXTRACTED", ""), + METASTORE_KERBEROS_KEYTAB_FILE("hive.metastore.kerberos.keytab.file", "", + "The path to the Kerberos Keytab file containing the metastore Thrift server's service principal."), METASTORE_KERBEROS_PRINCIPAL("hive.metastore.kerberos.principal", - "hive-metastore/_HOST@EXAMPLE.COM"), - METASTORE_USE_THRIFT_SASL("hive.metastore.sasl.enabled", false), - METASTORE_USE_THRIFT_FRAMED_TRANSPORT("hive.metastore.thrift.framed.transport.enabled", false), - METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS( - "hive.cluster.delegation.token.store.class", - "org.apache.hadoop.hive.thrift.MemoryTokenStore"), + "hive-metastore/_HOST@EXAMPLE.COM", + "The service principal for the metastore Thrift server. The special string _HOST will be replaced automatically with the correct host name."), + METASTORE_USE_THRIFT_SASL("hive.metastore.sasl.enabled", false, + "If true, the metastore Thrift interface will be secured with SASL. Clients must authenticate with Kerberos."), + METASTORE_USE_THRIFT_FRAMED_TRANSPORT("hive.metastore.thrift.framed.transport.enabled", false, + "If true, the metastore Thrift interface will use TFramedTransport. When false (default) a standard TTransport is used."), + METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS("hive.cluster.delegation.token.store.class", + "org.apache.hadoop.hive.thrift.MemoryTokenStore", + "The delegation token store implementation. Set to org.apache.hadoop.hive.thrift.ZooKeeperTokenStore for load-balanced cluster."), METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_CONNECTSTR( - "hive.cluster.delegation.token.store.zookeeper.connectString", ""), + "hive.cluster.delegation.token.store.zookeeper.connectString", "", + "The ZooKeeper token store connect string."), METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_ZNODE( - "hive.cluster.delegation.token.store.zookeeper.znode", "/hive/cluster/delegation"), + "hive.cluster.delegation.token.store.zookeeper.znode", "/hive/cluster/delegation", + "The root path for token store data."), METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_ZK_ACL( - "hive.cluster.delegation.token.store.zookeeper.acl", ""), - METASTORE_CACHE_PINOBJTYPES("hive.metastore.cache.pinobjtypes", "Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order"), - METASTORE_CONNECTION_POOLING_TYPE("datanucleus.connectionPoolingType", "BONECP"), - METASTORE_VALIDATE_TABLES("datanucleus.validateTables", false), - METASTORE_VALIDATE_COLUMNS("datanucleus.validateColumns", false), - METASTORE_VALIDATE_CONSTRAINTS("datanucleus.validateConstraints", false), - METASTORE_STORE_MANAGER_TYPE("datanucleus.storeManagerType", "rdbms"), - METASTORE_AUTO_CREATE_SCHEMA("datanucleus.autoCreateSchema", true), - METASTORE_FIXED_DATASTORE("datanucleus.fixedDatastore", false), - METASTORE_SCHEMA_VERIFICATION("hive.metastore.schema.verification", false), - METASTORE_AUTO_START_MECHANISM_MODE("datanucleus.autoStartMechanismMode", "checked"), - METASTORE_TRANSACTION_ISOLATION("datanucleus.transactionIsolation", "read-committed"), - METASTORE_CACHE_LEVEL2("datanucleus.cache.level2", false), - METASTORE_CACHE_LEVEL2_TYPE("datanucleus.cache.level2.type", "none"), - METASTORE_IDENTIFIER_FACTORY("datanucleus.identifierFactory", "datanucleus1"), - METASTORE_USE_LEGACY_VALUE_STRATEGY("datanucleus.rdbms.useLegacyNativeValueStrategy", true), - METASTORE_PLUGIN_REGISTRY_BUNDLE_CHECK("datanucleus.plugin.pluginRegistryBundleCheck", "LOG"), - METASTORE_BATCH_RETRIEVE_MAX("hive.metastore.batch.retrieve.max", 300), + "hive.cluster.delegation.token.store.zookeeper.acl", "", + "ACL for token store entries. List comma separated all server principals for the cluster."), + METASTORE_CACHE_PINOBJTYPES("hive.metastore.cache.pinobjtypes", "Table,StorageDescriptor,SerDeInfo,Partition,Database,Type,FieldSchema,Order", + "List of comma separated metastore object types that should be pinned in the cache"), + METASTORE_CONNECTION_POOLING_TYPE("datanucleus.connectionPoolingType", "BONECP", + "Specify connection pool library for datanucleus"), + METASTORE_VALIDATE_TABLES("datanucleus.validateTables", false, + "validates existing schema against code. turn this on if you want to verify existing schema"), + METASTORE_VALIDATE_COLUMNS("datanucleus.validateColumns", false, + "validates existing schema against code. turn this on if you want to verify existing schema"), + METASTORE_VALIDATE_CONSTRAINTS("datanucleus.validateConstraints", false, + "validates existing schema against code. turn this on if you want to verify existing schema"), + METASTORE_STORE_MANAGER_TYPE("datanucleus.storeManagerType", "rdbms", "metadata store type"), + METASTORE_AUTO_CREATE_SCHEMA("datanucleus.autoCreateSchema", true, + "creates necessary schema on a startup if one doesn't exist. set this to false, after creating it once"), + METASTORE_FIXED_DATASTORE("datanucleus.fixedDatastore", false, ""), + METASTORE_SCHEMA_VERIFICATION("hive.metastore.schema.verification", false, + "Enforce metastore schema version consistency.\n" + + "True: Verify that version information stored in metastore matches with one from Hive jars. Also disable automatic\n" + + " schema migration attempt. Users are required to manually migrate schema after Hive upgrade which ensures\n" + + " proper metastore schema migration. (Default)\n" + + "False: Warn if the version information stored in metastore doesn't match with one from in Hive jars."), + METASTORE_AUTO_START_MECHANISM_MODE("datanucleus.autoStartMechanismMode", "checked", + "throw exception if metadata tables are incorrect"), + METASTORE_TRANSACTION_ISOLATION("datanucleus.transactionIsolation", "read-committed", + "Default transaction isolation level for identity generation."), + METASTORE_CACHE_LEVEL2("datanucleus.cache.level2", false, + "Use a level 2 cache. Turn this off if metadata is changed independently of Hive metastore server"), + METASTORE_CACHE_LEVEL2_TYPE("datanucleus.cache.level2.type", "none", ""), + METASTORE_IDENTIFIER_FACTORY("datanucleus.identifierFactory", "datanucleus1", + "Name of the identifier factory to use when generating table/column names etc. \n" + + "'datanucleus1' is used for backward compatibility with DataNucleus v1"), + METASTORE_USE_LEGACY_VALUE_STRATEGY("datanucleus.rdbms.useLegacyNativeValueStrategy", true, ""), + METASTORE_PLUGIN_REGISTRY_BUNDLE_CHECK("datanucleus.plugin.pluginRegistryBundleCheck", "LOG", + "Defines what happens when plugin bundles are found and are duplicated [EXCEPTION|LOG|NONE]"), + METASTORE_BATCH_RETRIEVE_MAX("hive.metastore.batch.retrieve.max", 300, + "Maximum number of objects (tables/partitions) can be retrieved from metastore in one batch. \n" + + "The higher the number, the less the number of round trips is needed to the Hive metastore server, \n" + + "but it may also cause higher memory requirement at the client side."), METASTORE_BATCH_RETRIEVE_TABLE_PARTITION_MAX( - "hive.metastore.batch.retrieve.table.partition.max", 1000), - // A comma separated list of hooks which implement MetaStoreInitListener and will be run at - // the beginning of HMSHandler initialization - METASTORE_INIT_HOOKS("hive.metastore.init.hooks", ""), - METASTORE_PRE_EVENT_LISTENERS("hive.metastore.pre.event.listeners", ""), - METASTORE_EVENT_LISTENERS("hive.metastore.event.listeners", ""), - // should we do checks against the storage (usually hdfs) for operations like drop_partition - METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS("hive.metastore.authorization.storage.checks", false), - METASTORE_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq",0L), - METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration",0L), - METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", false), - METASTORE_PARTITION_NAME_WHITELIST_PATTERN( - "hive.metastore.partition.name.whitelist.pattern", ""), - // Whether to enable integral JDO pushdown. For partition columns storing integers - // in non-canonical form, (e.g. '012'), it may not work, so it's off by default. - METASTORE_INTEGER_JDO_PUSHDOWN("hive.metastore.integral.jdo.pushdown", false), - METASTORE_TRY_DIRECT_SQL("hive.metastore.try.direct.sql", true), - METASTORE_TRY_DIRECT_SQL_DDL("hive.metastore.try.direct.sql.ddl", true), - METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES( - "hive.metastore.disallow.incompatible.col.type.changes", false), + "hive.metastore.batch.retrieve.table.partition.max", 1000, + "Maximum number of table partitions that metastore internally retrieves in one batch."), - // Default parameters for creating tables - NEWTABLEDEFAULTPARA("hive.table.parameters.default", ""), - // Parameters to copy over when creating a table with Create Table Like. - DDL_CTL_PARAMETERS_WHITELIST("hive.ddl.createtablelike.properties.whitelist", ""), - METASTORE_RAW_STORE_IMPL("hive.metastore.rawstore.impl", - "org.apache.hadoop.hive.metastore.ObjectStore"), - METASTORE_CONNECTION_DRIVER("javax.jdo.option.ConnectionDriverName", - "org.apache.derby.jdbc.EmbeddedDriver"), + METASTORE_INIT_HOOKS("hive.metastore.init.hooks", "", + "A comma separated list of hooks to be invoked at the beginning of HMSHandler initialization. \n" + + "An init hook is specified as the name of Java class which extends org.apache.hadoop.hive.metastore.MetaStoreInitListener."), + METASTORE_PRE_EVENT_LISTENERS("hive.metastore.pre.event.listeners", "", + "List of comma separated listeners for metastore events."), + METASTORE_EVENT_LISTENERS("hive.metastore.event.listeners", "", ""), + METASTORE_AUTHORIZATION_STORAGE_AUTH_CHECKS("hive.metastore.authorization.storage.checks", false, + "Should the metastore do authorization checks against the underlying storage (usually hdfs) \n" + + "for operations like drop-partition (disallow the drop-partition if the user in\n" + + "question doesn't have permissions to delete the corresponding directory\n" + + "on the storage).\n"), + METASTORE_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq", 0L, + "Frequency at which timer task runs to purge expired events in metastore(in seconds)."), + METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration", 0L, + "Duration after which events expire from events table (in seconds)"), + METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", false, + "In unsecure mode, setting this property to true will cause the metastore to execute DFS operations using \n" + + "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 its best effort. \n" + + "If client sets its to true and server sets it to false, client setting will be ignored."), + METASTORE_PARTITION_NAME_WHITELIST_PATTERN("hive.metastore.partition.name.whitelist.pattern", "", + "Partition names will be checked against this regex pattern and rejected if not matched."), + + METASTORE_INTEGER_JDO_PUSHDOWN("hive.metastore.integral.jdo.pushdown", false, + "Allow JDO query pushdown for integral partition columns in metastore. Off by default. This\n" + + "improves metastore perf for integral columns, especially if there's a large number of partitions.\n" + + "However, it doesn't work correctly with integral values that are not normalized (e.g. have\n" + + "leading zeroes, like 0012). If metastore direct SQL is enabled and works, this optimization\n" + + "is also irrelevant."), + METASTORE_TRY_DIRECT_SQL("hive.metastore.try.direct.sql", true, ""), + METASTORE_TRY_DIRECT_SQL_DDL("hive.metastore.try.direct.sql.ddl", true, ""), + METASTORE_DISALLOW_INCOMPATIBLE_COL_TYPE_CHANGES( + "hive.metastore.disallow.incompatible.col.type.changes", false, + "If true (default is false), ALTER TABLE operations which change the type of \n" + + "a column (say STRING) to an incompatible type (say MAP<STRING, STRING>) are disallowed. \n" + + "RCFile default SerDe (ColumnarSerDe) serializes the values in such a way that the\n" + + "datatypes can be converted from string to any type. The map is also serialized as\n" + + "a string, which can be read as a string as well. However, with any binary \n" + + "serialization, this is not true. Blocking the ALTER TABLE prevents ClassCastExceptions\n" + + "when subsequently trying to access old partitions. \n" + + "\n" + + "Primitive types like INT, STRING, BIGINT, etc are compatible with each other and are \n" + + "not blocked. \n" + + "\n" + + "See HIVE-4409 for more details."), + + NEWTABLEDEFAULTPARA("hive.table.parameters.default", "", + "Default property values for newly created tables"), + DDL_CTL_PARAMETERS_WHITELIST("hive.ddl.createtablelike.properties.whitelist", "", + "Table Properties to copy over when executing a Create Table Like."), + METASTORE_RAW_STORE_IMPL("hive.metastore.rawstore.impl", "org.apache.hadoop.hive.metastore.ObjectStore", + "Name of the class that implements org.apache.hadoop.hive.metastore.rawstore interface. \n" + + "This class is used to store and retrieval of raw metadata objects such as table, database"), + METASTORE_CONNECTION_DRIVER("javax.jdo.option.ConnectionDriverName", "org.apache.derby.jdbc.EmbeddedDriver", + "Driver class name for a JDBC metastore"), METASTORE_MANAGER_FACTORY_CLASS("javax.jdo.PersistenceManagerFactoryClass", - "org.datanucleus.api.jdo.JDOPersistenceManagerFactory"), + "org.datanucleus.api.jdo.JDOPersistenceManagerFactory", + "class implementing the jdo persistence"), METASTORE_EXPRESSION_PROXY_CLASS("hive.metastore.expression.proxy", - "org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore"), - METASTORE_DETACH_ALL_ON_COMMIT("javax.jdo.option.DetachAllOnCommit", true), - METASTORE_NON_TRANSACTIONAL_READ("javax.jdo.option.NonTransactionalRead", true), - METASTORE_CONNECTION_USER_NAME("javax.jdo.option.ConnectionUserName", "APP"), - METASTORE_END_FUNCTION_LISTENERS("hive.metastore.end.function.listeners", ""), - METASTORE_PART_INHERIT_TBL_PROPS("hive.metastore.partition.inherit.table.properties",""), + "org.apache.hadoop.hive.ql.optimizer.ppr.PartitionExpressionForMetastore", ""), + METASTORE_DETACH_ALL_ON_COMMIT("javax.jdo.option.DetachAllOnCommit", true, + "Detaches all objects from session so that they can be used after transaction is committed"), + METASTORE_NON_TRANSACTIONAL_READ("javax.jdo.option.NonTransactionalRead", true, + "Reads outside of transactions"), + METASTORE_CONNECTION_USER_NAME("javax.jdo.option.ConnectionUserName", "APP", + "Username to use against metastore database"), + METASTORE_END_FUNCTION_LISTENERS("hive.metastore.end.function.listeners", "", + "List of comma separated listeners for the end of metastore functions."), + METASTORE_PART_INHERIT_TBL_PROPS("hive.metastore.partition.inherit.table.properties", "", + "List of comma separated keys occurring in table properties which will get inherited to newly created partitions. \n" + + "* implies all the keys will get inherited."), // Parameters for exporting metadata on table drop (requires the use of the) // org.apache.hadoop.hive.ql.parse.MetaDataExportListener preevent listener - METADATA_EXPORT_LOCATION("hive.metadata.export.location", ""), - MOVE_EXPORTED_METADATA_TO_TRASH("hive.metadata.move.exported.metadata.to.trash", true), + METADATA_EXPORT_LOCATION("hive.metadata.export.location", "", + "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" + + "it is the location to which the metadata will be exported. The default is an empty string, which results in the \n" + + "metadata being exported to the current user's home directory on HDFS."), + MOVE_EXPORTED_METADATA_TO_TRASH("hive.metadata.move.exported.metadata.to.trash", true, + "When used in conjunction with the org.apache.hadoop.hive.ql.parse.MetaDataExportListener pre event listener, \n" + + "this setting determines if the metadata that is exported will subsequently be moved to the user's trash directory \n" + + "alongside the dropped table data. This ensures that the metadata will be cleaned up along with the dropped table data."), // CLI - CLIIGNOREERRORS("hive.cli.errors.ignore", false), - CLIPRINTCURRENTDB("hive.cli.print.current.db", false), - CLIPROMPT("hive.cli.prompt", "hive"), - CLIPRETTYOUTPUTNUMCOLS("hive.cli.pretty.output.num.cols", -1), + CLIIGNOREERRORS("hive.cli.errors.ignore", false, ""), + CLIPRINTCURRENTDB("hive.cli.print.current.db", false, + "Whether to include the current database in the Hive prompt."), + CLIPROMPT("hive.cli.prompt", "hive", + "Command line prompt configuration value. Other hiveconf can be used in this configuration value. \n" + + "Variable substitution will only be invoked at the Hive CLI startup."), + CLIPRETTYOUTPUTNUMCOLS("hive.cli.pretty.output.num.cols", -1, + "The number of columns to use when formatting output generated by the DESCRIBE PRETTY table_name command.\n" + + "If the value of this property is -1, then Hive will use the auto-detected terminal width."), - HIVE_METASTORE_FS_HANDLER_CLS("hive.metastore.fs.handler.class", "org.apache.hadoop.hive.metastore.HiveMetaStoreFsImpl"), + HIVE_METASTORE_FS_HANDLER_CLS("hive.metastore.fs.handler.class", "org.apache.hadoop.hive.metastore.HiveMetaStoreFsImpl", ""), // Things we log in the jobconf // session identifier - HIVESESSIONID("hive.session.id", ""), + HIVESESSIONID("hive.session.id", "", ""), // whether session is running in silent mode or not - HIVESESSIONSILENT("hive.session.silent", false), + HIVESESSIONSILENT("hive.session.silent", false, ""), - // Whether to enable history for this session - HIVE_SESSION_HISTORY_ENABLED("hive.session.history.enabled", false), + HIVE_SESSION_HISTORY_ENABLED("hive.session.history.enabled", false, + "Whether to log Hive query, query plan, runtime statistics etc."), - // query being executed (multiple per session) - HIVEQUERYSTRING("hive.query.string", ""), + HIVEQUERYSTRING("hive.query.string", "", + "Query being executed (might be multiple per a session)"), - // id of query being executed (multiple per session) - HIVEQUERYID("hive.query.id", ""), + HIVEQUERYID("hive.query.id", "", + "ID for query being executed (might be multiple per a session)"), - // id of the mapred plan being executed (multiple per query) - HIVEPLANID("hive.query.planid", ""), - // max jobname length - HIVEJOBNAMELENGTH("hive.jobname.length", 50), + HIVEJOBNAMELENGTH("hive.jobname.length", 50, "max jobname length"), // hive jar - HIVEJAR("hive.jar.path", ""), - HIVEAUXJARS("hive.aux.jars.path", ""), + HIVEJAR("hive.jar.path", "", ""), + HIVEAUXJARS("hive.aux.jars.path", "", ""), // hive added files and jars - HIVEADDEDFILES("hive.added.files.path", ""), - HIVEADDEDJARS("hive.added.jars.path", ""), - HIVEADDEDARCHIVES("hive.added.archives.path", ""), + HIVEADDEDFILES("hive.added.files.path", "", ""), + HIVEADDEDJARS("hive.added.jars.path", "", ""), + HIVEADDEDARCHIVES("hive.added.archives.path", "", ""), - HIVE_CURRENT_DATABASE("hive.current.database", ""), // internal usage only + HIVE_CURRENT_DATABASE("hive.current.database", "", "current database in using. internal usage only", true), // for hive script operator - HIVES_AUTO_PROGRESS_TIMEOUT("hive.auto.progress.timeout", 0), - HIVETABLENAME("hive.table.name", ""), - HIVEPARTITIONNAME("hive.partition.name", ""), - HIVESCRIPTAUTOPROGRESS("hive.script.auto.progress", false), - HIVESCRIPTIDENVVAR("hive.script.operator.id.env.var", "HIVE_SCRIPT_OPERATOR_ID"), - HIVESCRIPTTRUNCATEENV("hive.script.operator.truncate.env", false), - HIVEMAPREDMODE("hive.mapred.mode", "nonstrict"), - HIVEALIAS("hive.alias", ""), - HIVEMAPSIDEAGGREGATE("hive.map.aggr", true), - HIVEGROUPBYSKEW("hive.groupby.skewindata", false), - HIVE_OPTIMIZE_MULTI_GROUPBY_COMMON_DISTINCTS("hive.optimize.multigroupby.common.distincts", - true), - HIVEJOINEMITINTERVAL("hive.join.emit.interval", 1000), - HIVEJOINCACHESIZE("hive.join.cache.size", 25000), + HIVES_AUTO_PROGRESS_TIMEOUT("hive.auto.progress.timeout", 0, + "How long to run autoprogressor for the script/UDTF operators (in seconds).\n" + + "Set to 0 for forever."), + HIVETABLENAME("hive.table.name", "", ""), + HIVEPARTITIONNAME("hive.partition.name", "", ""), + HIVESCRIPTAUTOPROGRESS("hive.script.auto.progress", false, + "Whether Hive Transform/Map/Reduce Clause should automatically send progress information to TaskTracker \n" + + "to avoid the task getting killed because of inactivity. Hive sends progress information when the script is \n" + + "outputting to stderr. This option removes the need of periodically producing stderr messages, \n" + + "but users should be cautious because this may prevent infinite loops in the scripts to be killed by TaskTracker."), + HIVESCRIPTIDENVVAR("hive.script.operator.id.env.var", "HIVE_SCRIPT_OPERATOR_ID", + "Name of the environment variable that holds the unique script operator ID in the user's \n" + + "transform function (the custom mapper/reducer that the user has specified in the query)"), + HIVESCRIPTTRUNCATEENV("hive.script.operator.truncate.env", false, + "Truncate each environment variable for external script in scripts operator to 20KB (to fit system limits)"), + HIVEMAPREDMODE("hive.mapred.mode", "nonstrict", + "The mode in which the Hive operations are being performed. \n" + + "In strict mode, some risky queries are not allowed to run. They include:\n" + + " Cartesian Product.\n" + + " No partition being picked up for a query.\n" + + " Comparing bigints and strings.\n" + + " Comparing bigints and doubles.\n" + + " Orderby without limit."), + HIVEALIAS("hive.alias", "", ""), + HIVEMAPSIDEAGGREGATE("hive.map.aggr", true, "Whether to use map-side aggregation in Hive Group By queries"), + HIVEGROUPBYSKEW("hive.groupby.skewindata", false, "Whether there is skew in data to optimize group by queries"), + HIVE_OPTIMIZE_MULTI_GROUPBY_COMMON_DISTINCTS("hive.optimize.multigroupby.common.distincts", true, + "Whether to optimize a multi-groupby query with the same distinct.\n" + + "Consider a query like:\n" + + "\n" + + " from src\n" + + " insert overwrite table dest1 select col1, count(distinct colx) group by col1\n" + + " insert overwrite table dest2 select col2, count(distinct colx) group by col2;\n" + + "\n" + + "With this parameter set to true, first we spray by the distinct value (colx), and then\n" + + "perform the 2 groups bys. This makes sense if map-side aggregation is turned off. However,\n" + + "with maps-side aggregation, it might be useful in some cases to treat the 2 inserts independently, \n" + + "thereby performing the query above in 2MR jobs instead of 3 (due to spraying by distinct key first).\n" + + "If this parameter is turned off, we don't consider the fact that the distinct key is the same across\n" + + "different MR jobs."), + HIVEJOINEMITINTERVAL("hive.join.emit.interval", 1000, + "How many rows in the right-most join operand Hive should buffer before emitting the join result."), + HIVEJOINCACHESIZE("hive.join.cache.size", 25000, + "How many rows in the joining tables (except the streaming table) should be cached in memory."), // hive.mapjoin.bucket.cache.size has been replaced by hive.smbjoin.cache.row, // need to remove by hive .13. Also, do not change default (see SMB operator) - HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100), + HIVEMAPJOINBUCKETCACHESIZE("hive.mapjoin.bucket.cache.size", 100, ""), - HIVESMBJOINCACHEROWS("hive.smbjoin.cache.rows", 10000), - HIVEGROUPBYMAPINTERVAL("hive.groupby.mapaggr.checkinterval", 100000), - HIVEMAPAGGRHASHMEMORY("hive.map.aggr.hash.percentmemory", (float) 0.5), - HIVEMAPJOINFOLLOWEDBYMAPAGGRHASHMEMORY("hive.mapjoin.followby.map.aggr.hash.percentmemory", (float) 0.3), - HIVEMAPAGGRMEMORYTHRESHOLD("hive.map.aggr.hash.force.flush.memory.threshold", (float) 0.9), - HIVEMAPAGGRHASHMINREDUCTION("hive.map.aggr.hash.min.reduction", (float) 0.5), - HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true), - HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false), - HIVE_MAP_GROUPBY_SORT_TESTMODE("hive.map.groupby.sorted.testmode", false), - HIVE_GROUPBY_ORDERBY_POSITION_ALIAS("hive.groupby.orderby.position.alias", false), - HIVE_NEW_JOB_GROUPING_SET_CARDINALITY("hive.new.job.grouping.set.cardinality", 30), + HIVESMBJOINCACHEROWS("hive.smbjoin.cache.rows", 10000, + "How many rows with the same key value should be cached in memory per smb joined table."), + HIVEGROUPBYMAPINTERVAL("hive.groupby.mapaggr.checkinterval", 100000, + "Number of rows after which size of the grouping keys/aggregation classes is performed"), + HIVEMAPAGGRHASHMEMORY("hive.map.aggr.hash.percentmemory", (float) 0.5, + "Portion of total memory to be used by map-side group aggregation hash table"), + HIVEMAPJOINFOLLOWEDBYMAPAGGRHASHMEMORY("hive.mapjoin.followby.map.aggr.hash.percentmemory", (float) 0.3, + "Portion of total memory to be used by map-side group aggregation hash table, when this group by is followed by map join"), + HIVEMAPAGGRMEMORYTHRESHOLD("hive.map.aggr.hash.force.flush.memory.threshold", (float) 0.9, + "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"), + HIVEMAPAGGRHASHMINREDUCTION("hive.map.aggr.hash.min.reduction", (float) 0.5, + "Hash aggregation will be turned off if the ratio between hash table size and input rows is bigger than this number. \n" + + "Set to 1 to make sure hash aggregation is never turned off."), + HIVEMULTIGROUPBYSINGLEREDUCER("hive.multigroupby.singlereducer", true, + "Whether to optimize multi group by query to generate single M/R job plan. If the multi group by query has \n" + + "common group by keys, it will be optimized to generate single M/R job."), + HIVE_MAP_GROUPBY_SORT("hive.map.groupby.sorted", false, + "If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform \n" + + "the group by in the mapper by using BucketizedHiveInputFormat. The only downside to this\n" + + "is that it limits the number of mappers to the number of files."), + HIVE_MAP_GROUPBY_SORT_TESTMODE("hive.map.groupby.sorted.testmode", false, + "If the bucketing/sorting properties of the table exactly match the grouping key, whether to perform \n" + + "the group by in the mapper by using BucketizedHiveInputFormat. If the test mode is set, the plan\n" + + "is not converted, but a query property is set to denote the same."), + HIVE_GROUPBY_ORDERBY_POSITION_ALIAS("hive.groupby.orderby.position.alias", false, + "Whether to enable using Column Position Alias in Group By or Order By"), + HIVE_NEW_JOB_GROUPING_SET_CARDINALITY("hive.new.job.grouping.set.cardinality", 30, + "Whether a new map-reduce job should be launched for grouping sets/rollups/cubes.\n" + + "For a query like: select a, b, c, count(1) from T group by a, b, c with rollup;\n" + + "4 rows are created per row: (a, b, c), (a, b, null), (a, null, null), (null, null, null).\n" + + "This can lead to explosion across map-reduce boundary if the cardinality of T is very high,\n" + + "and map-side aggregation does not do a very good job. \n" + + "\n" + + "This parameter decides if Hive should add an additional map-reduce job. If the grouping set\n" + + "cardinality (4 in the example above), is more than this value, a new MR job is added under the\n" + + "assumption that the original group by will reduce the data size."), // for hive udtf operator - HIVEUDTFAUTOPROGRESS("hive.udtf.auto.progress", false), - - // Default file format for CREATE TABLE statement - // Options: TextFile, SequenceFile - HIVEDEFAULTFILEFORMAT("hive.default.fileformat", "TextFile", - new StringsValidator("TextFile", "SequenceFile", "RCfile", "ORC")), - HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "TextFile", - new StringsValidator("TextFile", "SequenceFile", "RCfile")), - HIVECHECKFILEFORMAT("hive.fileformat.check", true), + HIVEUDTFAUTOPROGRESS("hive.udtf.auto.progress", false, + "Whether Hive should automatically send progress information to TaskTracker \n" + + "when using UDTF's to prevent the task getting killed because of inactivity. Users should be cautious \n" + + "because this may prevent TaskTracker from killing tasks with infinite loops."), + + HIVEDEFAULTFILEFORMAT("hive.default.fileformat", "TextFile", new StringSet("TextFile", "SequenceFile", "RCfile", "ORC"), + "Default file format for CREATE TABLE statement. \n" + + "Options are TextFile and SequenceFile. Users can explicitly say CREATE TABLE ... STORED AS to override"), + HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "TextFile", new StringSet("TextFile", "SequenceFile", "RCfile"), ""), + HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"), // default serde for rcfile HIVEDEFAULTRCFILESERDE("hive.default.rcfile.serde", - "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe"), + "org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe", + "The default SerDe Hive will use for the RCFile format"), - //Location of Hive run time structured log file - HIVEHISTORYFILELOC("hive.querylog.location", System.getProperty("java.io.tmpdir") + File.separator + System.getProperty("user.name")), + HIVEHISTORYFILELOC("hive.querylog.location", + System.getProperty("java.io.tmpdir") + File.separator + System.getProperty("user.name"), + "Location of Hive run time structured log file"), + + HIVE_LOG_INCREMENTAL_PLAN_PROGRESS("hive.querylog.enable.plan.progress", true, + "Whether to log the plan's progress every time a job's progress is checked.\n" + + "These logs are written to the location specified by hive.querylog.location"), + + HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL("hive.querylog.plan.progress.interval", 60000L, + "The interval to wait between logging the plan's progress in milliseconds.\n" + + "If there is a whole number percentage change in the progress of the mappers or the reducers,\n" + + "the progress is logged regardless of this value.\n" + + "The actual interval will be the ceiling of (this value divided by the value of\n" + + "hive.exec.counters.pull.interval) multiplied by the value of hive.exec.counters.pull.interval\n" + + "I.e. if it is not divide evenly by the value of hive.exec.counters.pull.interval it will be\n" + + "logged less frequently than specified.\n" + + "This only has an effect if hive.querylog.enable.plan.progress is set to true."), - // Whether to log the plan's progress every time a job's progress is checked - HIVE_LOG_INCREMENTAL_PLAN_PROGRESS("hive.querylog.enable.plan.progress", true), - - // The interval between logging the plan's progress in milliseconds - HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL("hive.querylog.plan.progress.interval", 60000L), - - // Default serde and record reader for user scripts - HIVESCRIPTSERDE("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"), + HIVESCRIPTSERDE("hive.script.serde", "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe", + "The default SerDe for transmitting input data to and reading output data from the user scripts. "), HIVESCRIPTRECORDREADER("hive.script.recordreader", - "org.apache.hadoop.hive.ql.exec.TextRecordReader"), + "org.apache.hadoop.hive.ql.exec.TextRecordReader", + "The default record reader for reading data from the user scripts. "), HIVESCRIPTRECORDWRITER("hive.script.recordwriter", - "org.apache.hadoop.hive.ql.exec.TextRecordWriter"), - HIVESCRIPTESCAPE("hive.transform.escape.input", false), - HIVEBINARYRECORDMAX("hive.binary.record.max.length", 1000 ), + "org.apache.hadoop.hive.ql.exec.TextRecordWriter", + "The default record writer for writing data to the user scripts. "), + HIVESCRIPTESCAPE("hive.transform.escape.input", false, + "This adds an option to escape special chars (newlines, carriage returns and\n" + + "tabs) when they are passed to the user script. This is useful if the Hive tables\n" + + "can contain data that contains special characters."), + HIVEBINARYRECORDMAX("hive.binary.record.max.length", 1000, + "Read from a binary stream and treat each hive.binary.record.max.length bytes as a record. \n" + + "The last record before the end of stream can have less than hive.binary.record.max.length bytes"), // HWI - HIVEHWILISTENHOST("hive.hwi.listen.host", "0.0.0.0"), - HIVEHWILISTENPORT("hive.hwi.listen.port", "9999"), - HIVEHWIWARFILE("hive.hwi.war.file", System.getenv("HWI_WAR_FILE")), + HIVEHWILISTENHOST("hive.hwi.listen.host", "0.0.0.0", "This is the host address the Hive Web Interface will listen on"), + HIVEHWILISTENPORT("hive.hwi.listen.port", "9999", "This is the port the Hive Web Interface will listen on"), + HIVEHWIWARFILE("hive.hwi.war.file", System.getenv("HWI_WAR_FILE"), + "This sets the path to the HWI war file, relative to ${HIVE_HOME}. "), - // mapper/reducer memory in local mode - HIVEHADOOPMAXMEM("hive.mapred.local.mem", 0), + HIVEHADOOPMAXMEM("hive.mapred.local.mem", 0, "mapper/reducer memory in local mode"), //small table file size - HIVESMALLTABLESFILESIZE("hive.mapjoin.smalltable.filesize",25000000L), //25M + HIVESMALLTABLESFILESIZE("hive.mapjoin.smalltable.filesize", 25000000L, + "The threshold for the input file size of the small tables; if the file size is smaller \n" + + "than this threshold, it will try to convert the common join into map join"), - // random number for split sampling - HIVESAMPLERANDOMNUM("hive.sample.seednumber", 0), + HIVESAMPLERANDOMNUM("hive.sample.seednumber", 0, + "A number used to percentage sampling. By changing this number, user will change the subsets of data sampled."), // test mode in hive mode - HIVETESTMODE("hive.test.mode", false), - HIVETESTMODEPREFIX("hive.test.mode.prefix", "test_"), - HIVETESTMODESAMPLEFREQ("hive.test.mode.samplefreq", 32), - HIVETESTMODENOSAMPLE("hive.test.mode.nosamplelist", ""), - - HIVEMERGEMAPFILES("hive.merge.mapfiles", true), - HIVEMERGEMAPREDFILES("hive.merge.mapredfiles", false), - HIVEMERGEMAPFILESSIZE("hive.merge.size.per.task", (long) (256 * 1000 * 1000)), - HIVEMERGEMAPFILESAVGSIZE("hive.merge.smallfiles.avgsize", (long) (16 * 1000 * 1000)), - HIVEMERGERCFILEBLOCKLEVEL("hive.merge.rcfile.block.level", true), + HIVETESTMODE("hive.test.mode", false, + "Whether Hive is running in test mode. If yes, it turns on sampling and prefixes the output tablename."), + HIVETESTMODEPREFIX("hive.test.mode.prefix", "test_", + "In test mode, specfies prefixes for the output table"), + HIVETESTMODESAMPLEFREQ("hive.test.mode.samplefreq", 32, + "In test mode, specfies sampling frequency for table, which is not bucketed,\n" + + "For example, the following query:" + + " INSERT OVERWRITE TABLE dest" + + " SELECT col1 from src" + + "would be converted to" + + " INSERT OVERWRITE TABLE test_dest" + + " SELECT col1 from src TABLESAMPLE (BUCKET 1 out of 32 on rand(1))"), + HIVETESTMODENOSAMPLE("hive.test.mode.nosamplelist", "", + "In test mode, specifies comma separated table names which would not apply sampling"), + + HIVEMERGEMAPFILES("hive.merge.mapfiles", true, + "Merge small files at the end of a map-only job"), + HIVEMERGEMAPREDFILES("hive.merge.mapredfiles", false, + "Merge small files at the end of a map-reduce job"), + HIVEMERGEMAPFILESSIZE("hive.merge.size.per.task", (long) (256 * 1000 * 1000), + "Size of merged files at the end of the job"), + HIVEMERGEMAPFILESAVGSIZE("hive.merge.smallfiles.avgsize", (long) (16 * 1000 * 1000), + "When the average output file size of a job is less than this number, Hive will start an additional \n" + + "map-reduce job to merge the output files into bigger files. This is only done for map-only jobs \n" + + "if hive.merge.mapfiles is true, and for map-reduce jobs if hive.merge.mapredfiles is true."), + HIVEMERGERCFILEBLOCKLEVEL("hive.merge.rcfile.block.level", true, ""), HIVEMERGEINPUTFORMATBLOCKLEVEL("hive.merge.input.format.block.level", - "org.apache.hadoop.hive.ql.io.rcfile.merge.RCFileBlockMergeInputFormat"), + "org.apache.hadoop.hive.ql.io.rcfile.merge.RCFileBlockMergeInputFormat", ""), HIVEMERGECURRENTJOBHASDYNAMICPARTITIONS( - "hive.merge.current.job.has.dynamic.partitions", false), - - HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true), - HIVEUSERCFILESYNCCACHE("hive.exec.rcfile.use.sync.cache", true), + "hive.merge.current.job.has.dynamic.partitions", false, ""), - // Maximum fraction of heap that can be used by ORC file writers - HIVE_ORC_FILE_MEMORY_POOL("hive.exec.orc.memory.pool", 0.5f), // 50% - // Define the version of the file to write - HIVE_ORC_WRITE_FORMAT("hive.exec.orc.write.format", null), - // Define the default ORC stripe size + HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true, + "If this is set the header for RCFiles will simply be RCF. If this is not\n" + + "set the header will be that borrowed from sequence files, e.g. SEQ- followed\n" + + "by the input and output RCFile formats."), + HIVEUSERCFILESYNCCACHE("hive.exec.rcfile.use.sync.cache", true, ""), + + HIVE_ORC_FILE_MEMORY_POOL("hive.exec.orc.memory.pool", 0.5f, + "Maximum fraction of heap that can be used by ORC file writers"), + HIVE_ORC_WRITE_FORMAT("hive.exec.orc.write.format", null, + "Define the version of the file to write"), HIVE_ORC_DEFAULT_STRIPE_SIZE("hive.exec.orc.default.stripe.size", - 256L * 1024 * 1024), - HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD( - "hive.exec.orc.dictionary.key.size.threshold", 0.8f), - // Define the default ORC index stride - HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride" - , 10000), - // Define the default ORC buffer size - HIVE_ORC_DEFAULT_BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024), - // Define the default block padding - HIVE_ORC_DEFAULT_BLOCK_PADDING("hive.exec.orc.default.block.padding", - true), - // Define the default compression codec for ORC file - HIVE_ORC_DEFAULT_COMPRESS("hive.exec.orc.default.compress", "ZLIB"), - - HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS("hive.orc.splits.include.file.footer", false), - HIVE_ORC_CACHE_STRIPE_DETAILS_SIZE("hive.orc.cache.stripe.details.size", 10000), - HIVE_ORC_COMPUTE_SPLITS_NUM_THREADS("hive.orc.compute.splits.num.threads", 10), - - HIVESKEWJOIN("hive.optimize.skewjoin", false), - HIVECONVERTJOIN("hive.auto.convert.join", true), - HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true), + 256L * 1024 * 1024, + "Define the default ORC stripe size"), + + HIVE_ORC_DICTIONARY_KEY_SIZE_THRESHOLD("hive.exec.orc.dictionary.key.size.threshold", 0.8f, + "If the number of keys in a dictionary is greater than this fraction of the total number of\n" + + "non-null rows, turn off dictionary encoding. Use 1 to always use dictionary encoding."), + HIVE_ORC_DEFAULT_ROW_INDEX_STRIDE("hive.exec.orc.default.row.index.stride", 10000, "Define the default ORC index stride"), + HIVE_ORC_DEFAULT_BUFFER_SIZE("hive.exec.orc.default.buffer.size", 256 * 1024, "Define the default ORC buffer size"), + HIVE_ORC_DEFAULT_BLOCK_PADDING("hive.exec.orc.default.block.padding", true, "Define the default block padding"), + HIVE_ORC_DEFAULT_COMPRESS("hive.exec.orc.default.compress", "ZLIB", "Define the default compression codec for ORC file"), + + HIVE_ORC_INCLUDE_FILE_FOOTER_IN_SPLITS("hive.orc.splits.include.file.footer", false, + "If turned on splits generated by orc will include metadata about the stripes in the file. This\n" + + "data is read remotely (from the client or HS2 machine) and sent to all the tasks."), + HIVE_ORC_CACHE_STRIPE_DETAILS_SIZE("hive.orc.cache.stripe.details.size", 10000, + "Cache size for keeping meta info about orc splits cached in the client."), + HIVE_ORC_COMPUTE_SPLITS_NUM_THREADS("hive.orc.compute.splits.num.threads", 10, + "How many threads orc should use to create splits in parallel."), + + HIVESKEWJOIN("hive.optimize.skewjoin", false, + "Whether to enable skew join optimization. \n" + + "The algorithm is as follows: At runtime, detect the keys with a large skew. Instead of\n" + + "processing those keys, store them temporarily in an HDFS directory. In a follow-up map-reduce\n" + + "job, process those skewed keys. The same key need not be skewed for all the tables, and so,\n" + + "the follow-up map-reduce job (for the skewed keys) would be much faster, since it would be a\n" + + "map-join."), + HIVECONVERTJOIN("hive.auto.convert.join", true, + "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size"), + HIVECONVERTJOINNOCONDITIONALTASK("hive.auto.convert.join.noconditionaltask", true, + "Whether Hive enables the optimization about converting common join into mapjoin based on the input file size. \n" + + "If this parameter is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than the\n" + + "specified size, the join is directly converted to a mapjoin (there is no conditional task)."), HIVECONVERTJOINNOCONDITIONALTASKTHRESHOLD("hive.auto.convert.join.noconditionaltask.size", - 10000000L), - HIVECONVERTJOINUSENONSTAGED("hive.auto.convert.join.use.nonstaged", true), - HIVESKEWJOINKEY("hive.skewjoin.key", 100000), - HIVESKEWJOINMAPJOINNUMMAPTASK("hive.skewjoin.mapjoin.map.tasks", 10000), - HIVESKEWJOINMAPJOINMINSPLIT("hive.skewjoin.mapjoin.min.split", 33554432L), //32M - - HIVESENDHEARTBEAT("hive.heartbeat.interval", 1000), - HIVELIMITMAXROWSIZE("hive.limit.row.max.size", 100000L), - HIVELIMITOPTLIMITFILE("hive.limit.optimize.limit.file", 10), - HIVELIMITOPTENABLE("hive.limit.optimize.enable", false), - HIVELIMITOPTMAXFETCH("hive.limit.optimize.fetch.max", 50000), - HIVELIMITPUSHDOWNMEMORYUSAGE("hive.limit.pushdown.memory.usage", -1f), - - HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000), - HIVEHASHTABLELOADFACTOR("hive.hashtable.loadfactor", (float) 0.75), - HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55), - HIVEHASHTABLEMAXMEMORYUSAGE("hive.mapjoin.localtask.max.memory.usage", (float) 0.90), - HIVEHASHTABLESCALE("hive.mapjoin.check.memory.rows", (long)100000), - - HIVEDEBUGLOCALTASK("hive.debug.localtask",false), - - HIVEINPUTFORMAT("hive.input.format", "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat"), - - HIVEENFORCEBUCKETING("hive.enforce.bucketing", false), - HIVEENFORCESORTING("hive.enforce.sorting", false), - HIVEOPTIMIZEBUCKETINGSORTING("hive.optimize.bucketingsorting", true), - HIVEPARTITIONER("hive.mapred.partitioner", "org.apache.hadoop.hive.ql.io.DefaultHivePartitioner"), - HIVEENFORCESORTMERGEBUCKETMAPJOIN("hive.enforce.sortmergebucketmapjoin", false), - HIVEENFORCEBUCKETMAPJOIN("hive.enforce.bucketmapjoin", false), + 10000000L, + "If hive.auto.convert.join.noconditionaltask is off, this parameter does not take affect. \n" + + "However, if it is on, and the sum of size for n-1 of the tables/partitions for a n-way join is smaller than this size, \n" + + "the join is directly converted to a mapjoin(there is no conditional task). The default is 10MB"), + HIVECONVERTJOINUSENONSTAGED("hive.auto.convert.join.use.nonstaged", true, + "For conditional joins, if input stream from a small alias can be directly applied to join operator without \n" + + "filtering or projection, the alias need not to be pre-staged in distributed cache via mapred local task.\n" + + "Currently, this is not working with vectorization or tez execution engine."), + HIVESKEWJOINKEY("hive.skewjoin.key", 100000, + "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,\n" + + "we think the key as a skew join key. "), + HIVESKEWJOINMAPJOINNUMMAPTASK("hive.skewjoin.mapjoin.map.tasks", 10000, + "Determine the number of map task used in the follow up map join job for a skew join.\n" + + "It should be used together with hive.skewjoin.mapjoin.min.split to perform a fine grained control."), + HIVESKEWJOINMAPJOINMINSPLIT("hive.skewjoin.mapjoin.min.split", 33554432L, + "Determine the number of map task at most used in the follow up map join job for a skew join by specifying \n" + + "the minimum split size. It should be used together with hive.skewjoin.mapjoin.map.tasks to perform a fine grained control."), + + HIVESENDHEARTBEAT("hive.heartbeat.interval", 1000, + "Send a heartbeat after this interval - used by mapjoin and filter operators"), + HIVELIMITMAXROWSIZE("hive.limit.row.max.size", 100000L, + "When trying a smaller subset of data for simple LIMIT, how much size we need to guarantee each row to have at least."), + HIVELIMITOPTLIMITFILE("hive.limit.optimize.limit.file", 10, + "When trying a smaller subset of data for simple LIMIT, maximum number of files we can sample."), + HIVELIMITOPTENABLE("hive.limit.optimize.enable", false, + "Whether to enable to optimization to trying a smaller subset of data for simple LIMIT first."), + HIVELIMITOPTMAXFETCH("hive.limit.optimize.fetch.max", 50000, + "Maximum number of rows allowed for a smaller subset of data for simple LIMIT, if it is a fetch query. \n" + + "Insert queries are not restricted by this limit."), + HIVELIMITPUSHDOWNMEMORYUSAGE("hive.limit.pushdown.memory.usage", -1f, + "The max memory to be used for hash in RS operator for top K selection."), + + HIVEHASHTABLETHRESHOLD("hive.hashtable.initialCapacity", 100000, ""), + HIVEHASHTABLELOADFACTOR("hive.hashtable.loadfactor", (float) 0.75, ""), + HIVEHASHTABLEFOLLOWBYGBYMAXMEMORYUSAGE("hive.mapjoin.followby.gby.localtask.max.memory.usage", (float) 0.55, + "This number means how much memory the local task can take to hold the key/value into an in-memory hash table \n" + + "when this map join is followed by a group by. If the local task's memory usage is more than this number, \n" + + "the local task will abort by itself. It means the data of the small table is too large to be held in memory."), + HIVEHASHTABLEMAXMEMORYUSAGE("hive.mapjoin.localtask.max.memory.usage", (float) 0.90, + "This number means how much memory the local task can take to hold the key/value into an in-memory hash table. \n" + + "If the local task's memory usage is more than this number, the local task will abort by itself. \n" + + "It means the data of the small table is too large to be held in memory."), + HIVEHASHTABLESCALE("hive.mapjoin.check.memory.rows", (long)100000, + "The number means after how many rows processed it needs to check the memory usage"), + + HIVEDEBUGLOCALTASK("hive.debug.localtask",false, ""), + + HIVEINPUTFORMAT("hive.input.format", "org.apache.hadoop.hive.ql.io.CombineHiveInputFormat", + "The default input format. Set this to HiveInputFormat if you encounter problems with CombineHiveInputFormat."), + + HIVEENFORCEBUCKETING("hive.enforce.bucketing", false, + "Whether bucketing is enforced. If true, while inserting into the table, bucketing is enforced."), + HIVEENFORCESORTING("hive.enforce.sorting", false, + "Whether sorting is enforced. If true, while inserting into the table, sorting is enforced."), + HIVEOPTIMIZEBUCKETINGSORTING("hive.optimize.bucketingsorting", true, + "If hive.enforce.bucketing or hive.enforce.sorting is true, don't create a reducer for enforcing \n" + + "bucketing/sorting for queries of the form: \n" + + "insert overwrite table T2 select * from T1;\n" + + "where T1 and T2 are bucketed/sorted by the same keys into the same number of buckets."), + HIVEPARTITIONER("hive.mapred.partitioner", "org.apache.hadoop.hive.ql.io.DefaultHivePartitioner", ""), + HIVEENFORCESORTMERGEBUCKETMAPJOIN("hive.enforce.sortmergebucketmapjoin", false, + "If the user asked for sort-merge bucketed map-side join, and it cannot be performed, should the query fail or not ?"), + HIVEENFORCEBUCKETMAPJOIN("hive.enforce.bucketmapjoin", false, + "If the user asked for bucketed map-side join, and it cannot be performed, \n" + + "should the query fail or not ? For example, if the buckets in the tables being joined are\n" + + "not a multiple of each other, bucketed map-side join cannot be performed, and the\n" + + "query will fail if hive.enforce.bucketmapjoin is set to true."), - HIVE_AUTO_SORTMERGE_JOIN("hive.auto.convert.sortmerge.join", false), + HIVE_AUTO_SORTMERGE_JOIN("hive.auto.convert.sortmerge.join", false, + "Will the join be automatically converted to a sort-merge join, if the joined tables pass the criteria for sort-merge join."), HIVE_AUTO_SORTMERGE_JOIN_BIGTABLE_SELECTOR( "hive.auto.convert.sortmerge.join.bigtable.selection.policy", - "org.apache.hadoop.hive.ql.optimizer.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ"), + "org.apache.hadoop.hive.ql.optimizer.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ", + "The policy to choose the big table for automatic conversion to sort-merge join. \n" + + "By default, the table with the largest partitions is assigned the big table. All policies are:\n" + + ". based on position of the table - the leftmost table is selected\n" + + "org.apache.hadoop.hive.ql.optimizer.LeftmostBigTableSMJ.\n" + + ". based on total size (all the partitions selected in the query) of the table \n" + + "org.apache.hadoop.hive.ql.optimizer.TableSizeBasedBigTableSelectorForAutoSMJ.\n" + + ". based on average size (all the partitions selected in the query) of the table \n" + + "org.apache.hadoop.hive.ql.optimizer.AvgPartitionSizeBasedBigTableSelectorForAutoSMJ.\n" + + "New policies can be added in future."), HIVE_AUTO_SORTMERGE_JOIN_TOMAPJOIN( - "hive.auto.convert.sortmerge.join.to.mapjoin", false), - - HIVESCRIPTOPERATORTRUST("hive.exec.script.trust", false), - HIVEROWOFFSET("hive.exec.rowoffset", false), + "hive.auto.convert.sortmerge.join.to.mapjoin", false, + "If hive.auto.convert.sortmerge.join is set to true, and a join was converted to a sort-merge join, \n" + + "this parameter decides whether each table should be tried as a big table, and effectively a map-join should be\n" + + "tried. That would create a conditional task with n+1 children for a n-way join (1 child for each table as the\n" + + "big table), and the backup task will be the sort-merge join. In some cases, a map-join would be faster than a\n" + + "sort-merge join, if there is no advantage of having the output bucketed and sorted. For example, if a very big sorted\n" + + "and bucketed table with few files (say 10 files) are being joined with a very small sorter and bucketed table\n" + + "with few files (10 files), the sort-merge join will only use 10 mappers, and a simple map-only join might be faster\n" + + "if the complete small table can fit in memory, and a map-join can be performed."), + + HIVESCRIPTOPERATORTRUST("hive.exec.script.trust", false, ""), + HIVEROWOFFSET("hive.exec.rowoffset", false, + "Whether to provide the row offset virtual column"), - HIVE_COMBINE_INPUT_FORMAT_SUPPORTS_SPLITTABLE("hive.hadoop.supports.splittable.combineinputformat", false), + HIVE_COMBINE_INPUT_FORMAT_SUPPORTS_SPLITTABLE("hive.hadoop.supports.splittable.combineinputformat", false, ""), // Optimizer - HIVEOPTINDEXFILTER("hive.optimize.index.filter", false), // automatically use indexes - HIVEINDEXAUTOUPDATE("hive.optimize.index.autoupdate", false), //automatically update stale indexes - HIVEOPTPPD("hive.optimize.ppd", true), // predicate pushdown - HIVEPPDRECOGNIZETRANSITIVITY("hive.ppd.recognizetransivity", true), // predicate pushdown - HIVEPPDREMOVEDUPLICATEFILTERS("hive.ppd.remove.duplicatefilters", true), - HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", true), - // push predicates down to storage handlers - HIVEOPTPPD_STORAGE("hive.optimize.ppd.storage", true), - HIVEOPTGROUPBY("hive.optimize.groupby", true), // optimize group by - HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false), // optimize bucket map join - HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false), // try to use sorted merge bucket map join - HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true), - HIVEOPTREDUCEDEDUPLICATIONMINREDUCER("hive.optimize.reducededuplication.min.reducer", 4), - - HIVESAMPLINGFORORDERBY("hive.optimize.sampling.orderby", false), - HIVESAMPLINGNUMBERFORORDERBY("hive.optimize.sampling.orderby.number", 1000), - HIVESAMPLINGPERCENTFORORDERBY("hive.optimize.sampling.orderby.percent", 0.1f), + HIVEOPTINDEXFILTER("hive.optimize.index.filter", false, + "Whether to enable automatic use of indexes"), + HIVEINDEXAUTOUPDATE("hive.optimize.index.autoupdate", false, + "Whether to update stale indexes automatically"), + HIVEOPTPPD("hive.optimize.ppd", true, + "Whether to enable predicate pushdown"), + HIVEPPDRECOGNIZETRANSITIVITY("hive.ppd.recognizetransivity", true, + "Whether to transitively replicate predicate filters over equijoin conditions."), + HIVEPPDREMOVEDUPLICATEFILTERS("hive.ppd.remove.duplicatefilters", true, + "Whether to push predicates down into storage handlers. Ignored when hive.optimize.ppd is false."), + HIVEMETADATAONLYQUERIES("hive.optimize.metadataonly", true, ""), + HIVEOPTPPD_STORAGE("hive.optimize.ppd.storage", true, + "Whether to push predicates down to storage handlers"), + HIVEOPTGROUPBY("hive.optimize.groupby", true, + "Whether to enable the bucketed group by from bucketed partitions/tables."), + HIVEOPTBUCKETMAPJOIN("hive.optimize.bucketmapjoin", false, + "Whether to try bucket mapjoin"), + HIVEOPTSORTMERGEBUCKETMAPJOIN("hive.optimize.bucketmapjoin.sortedmerge", false, + "Whether to try sorted bucket merge map join"), + HIVEOPTREDUCEDEDUPLICATION("hive.optimize.reducededuplication", true, + "Remove extra map-reduce jobs if the data is already clustered by the same key which needs to be used again. \n" + + "This should always be set to true. Since it is a new feature, it has been made configurable."), + HIVEOPTREDUCEDEDUPLICATIONMINREDUCER("hive.optimize.reducededuplication.min.reducer", 4, + "Reduce deduplication merges two RSs by moving key/parts/reducer-num of the child RS to parent RS. \n" + + "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.\n" + + "The optimization will be automatically disabled if number of reducers would be less than specified value."), + + HIVESAMPLINGFORORDERBY("hive.optimize.sampling.orderby", false, ""), + HIVESAMPLINGNUMBERFORORDERBY("hive.optimize.sampling.orderby.number", 1000, ""), + HIVESAMPLINGPERCENTFORORDERBY("hive.optimize.sampling.orderby.percent", 0.1f, ""), // whether to optimize union followed by select followed by filesink // It creates sub-directories in the final output, so should not be turned on in systems // where MAPREDUCE-1501 is not present - HIVE_OPTIMIZE_UNION_REMOVE("hive.optimize.union.remove", false), - HIVEOPTCORRELATION("hive.optimize.correlation", false), // exploit intra-query correlations - - // whether hadoop map-reduce supports sub-directories. It was added by MAPREDUCE-1501. - // Some optimizations can only be performed if the version of hadoop being used supports - // sub-directories - HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES("hive.mapred.supports.subdirectories", false), - - // optimize skewed join by changing the query plan at compile time - HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME("hive.optimize.skewjoin.compiletime", false), + HIVE_OPTIMIZE_UNION_REMOVE("hive.optimize.union.remove", false, + "Whether to remove the union and push the operators between union and the filesink above union. \n" + + "This avoids an extra scan of the output by union. This is independently useful for union\n" + + "queries, and specially useful when hive.optimize.skewjoin.compiletime is set to true, since an\n" + + "extra union is inserted.\n" + + "\n" + + "The merge is triggered if either of hive.merge.mapfiles or hive.merge.mapredfiles is set to true.\n" + + "If the user has set hive.merge.mapfiles to true and hive.merge.mapredfiles to false, the idea was the\n" + + "number of reducers are few, so the number of files anyway are small. However, with this optimization,\n" + + "we are increasing the number of files possibly by a big margin. So, we merge aggressively."), + HIVEOPTCORRELATION("hive.optimize.correlation", false, "exploit intra-query correlations."), + + HIVE_HADOOP_SUPPORTS_SUBDIRECTORIES("hive.mapred.supports.subdirectories", false, + "Whether the version of Hadoop which is running supports sub-directories for tables/partitions. \n" + + "Many Hive optimizations can be applied if the Hadoop version supports sub-directories for\n" + + "tables/partitions. It was added by MAPREDUCE-1501"), + + HIVE_OPTIMIZE_SKEWJOIN_COMPILETIME("hive.optimize.skewjoin.compiletime", false, + "Whether to create a separate plan for skewed keys for the tables in the join.\n" + + "This is based on the skewed keys stored in the metadata. At compile time, the plan is broken\n" + + "into different joins: one for the skewed keys, and the other for the remaining keys. And then,\n" + + "a union is performed for the 2 joins generated above. So unless the same skewed key is present\n" + + "in both the joined tables, the join for the skewed key will be performed as a map-side join.\n" + + "\n" + + "The main difference between this parameter and hive.optimize.skewjoin is that this parameter\n" + + "uses the skew information stored in the metastore to optimize the plan at compile time itself.\n" + + "If there is no skew information in the metadata, this parameter will not have any affect.\n" + + "Both hive.optimize.skewjoin.compiletime and hive.optimize.skewjoin should be set to true.\n" + + "Ideally, hive.optimize.skewjoin should be renamed as hive.optimize.skewjoin.runtime, but not doing\n" + + "so for backward compatibility.\n" + + "\n" + + "If the skew information is correctly stored in the metadata, hive.optimize.skewjoin.compiletime\n" + + "would change the query plan to take care of it, and hive.optimize.skewjoin will be a no-op."), // Indexes - HIVEOPTINDEXFILTER_COMPACT_MINSIZE("hive.optimize.index.filter.compact.minsize", (long) 5 * 1024 * 1024 * 1024), // 5G - HIVEOPTINDEXFILTER_COMPACT_MAXSIZE("hive.optimize.index.filter.compact.maxsize", (long) -1), // infinity - HIVE_INDEX_COMPACT_QUERY_MAX_ENTRIES("hive.index.compact.query.max.entries", (long) 10000000), // 10M - HIVE_INDEX_COMPACT_QUERY_MAX_SIZE("hive.index.compact.query.max.size", (long) 10 * 1024 * 1024 * 1024), // 10G - HIVE_INDEX_COMPACT_BINARY_SEARCH("hive.index.compact.binary.search", true), + HIVEOPTINDEXFILTER_COMPACT_MINSIZE("hive.optimize.index.filter.compact.minsize", (long) 5 * 1024 * 1024 * 1024, + "Minimum size (in bytes) of the inputs on which a compact index is automatically used."), // 5G + HIVEOPTINDEXFILTER_COMPACT_MAXSIZE("hive.optimize.index.filter.compact.maxsize", (long) -1, + "Maximum size (in bytes) of the inputs on which a compact index is automatically used. A negative number is equivalent to infinity."), // infinity + HIVE_INDEX_COMPACT_QUERY_MAX_ENTRIES("hive.index.compact.query.max.entries", (long) 10000000, + "The maximum number of index entries to read during a query that uses the compact index. Negative value is equivalent to infinity."), // 10M + HIVE_INDEX_COMPACT_QUERY_MAX_SIZE("hive.index.compact.query.max.size", (long) 10 * 1024 * 1024 * 1024, + "The maximum number of bytes that a query using the compact index can read. Negative value is equivalent to infinity."), // 10G + HIVE_INDEX_COMPACT_BINARY_SEARCH("hive.index.compact.binary.search", true, + "Whether or not to use a binary search to find the entries in an index table that match the filter, where possible"), // Statistics - HIVESTATSAUTOGATHER("hive.stats.autogather", true), - HIVESTATSDBCLASS("hive.stats.dbclass", "counter", - new PatternValidator("jdbc(:.*)", "hbase", "counter", "custom")), // StatsSetupConst.StatDB + HIVESTATSAUTOGATHER("hive.stats.autogather", true, + "A flag to gather statistics automatically during the INSERT OVERWRITE command."), + HIVESTATSDBCLASS("hive.stats.dbclass", "counter", new PatternSet("jdbc(:.*)", "hbase", "counter", "custom"), + "The storage that stores temporary Hive statistics. Currently, jdbc, hbase, counter and custom type are supported." + ), // StatsSetupConst.StatDB HIVESTATSJDBCDRIVER("hive.stats.jdbcdriver", - "org.apache.derby.jdbc.EmbeddedDriver"), // JDBC driver specific to the dbclass + "org.apache.derby.jdbc.EmbeddedDriver", + "The JDBC driver for the database that stores temporary Hive statistics."), HIVESTATSDBCONNECTIONSTRING("hive.stats.dbconnectionstring", - "jdbc:derby:;databaseName=TempStatsStore;create=true"), // automatically create database - HIVE_STATS_DEFAULT_PUBLISHER("hive.stats.default.publisher", - ""), // default stats publisher if none of JDBC/HBase is specified - HIVE_STATS_DEFAULT_AGGREGATOR("hive.stats.default.aggregator", - ""), // default stats aggregator if none of JDBC/HBase is specified - HIVE_STATS_JDBC_TIMEOUT("hive.stats.jdbc.timeout", - 30), // default timeout in sec for JDBC connection & SQL statements - HIVE_STATS_ATOMIC("hive.stats.atomic", - false), // whether to update metastore stats only if all stats are available - HIVE_STATS_RETRIES_MAX("hive.stats.retries.max", - 0), // maximum # of retries to insert/select/delete the stats DB - HIVE_STATS_RETRIES_WAIT("hive.stats.retries.wait", - 3000), // # milliseconds to wait before the next retry - HIVE_STATS_COLLECT_RAWDATASIZE("hive.stats.collect.rawdatasize", true), + "jdbc:derby:;databaseName=TempStatsStore;create=true", + "The default connection string for the database that stores temporary Hive statistics."), // automatically create database + HIVE_STATS_DEFAULT_PUBLISHER("hive.stats.default.publisher", "", + "The Java class (implementing the StatsPublisher interface) that is used by default if hive.stats.dbclass is custom type."), + HIVE_STATS_DEFAULT_AGGREGATOR("hive.stats.default.aggregator", "", + "The Java class (implementing the StatsAggregator interface) that is used by default if hive.stats.dbclass is custom type."), + HIVE_STATS_JDBC_TIMEOUT("hive.stats.jdbc.timeout", 30, + "Timeout value (number of seconds) used by JDBC connection and statements."), + HIVE_STATS_ATOMIC("hive.stats.atomic", false, + "whether to update metastore stats only if all stats are available"), + HIVE_STATS_RETRIES_MAX("hive.stats.retries.max", 0, + "Maximum number of retries when stats publisher/aggregator got an exception updating intermediate database. \n" + + "Default is no tries on failures."), + HIVE_STATS_RETRIES_WAIT("hive.stats.retries.wait", 3000, + "The base waiting window (in milliseconds) before the next retry. The actual wait time is calculated by " + [... 1006 lines stripped ...]