You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by na...@apache.org on 2014/09/02 06:41:31 UTC

svn commit: r1621912 [1/2] - in /hive/trunk: common/src/java/org/apache/hadoop/hive/conf/ hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/ hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/ itests/hive-unit/src/test/java/org/apach...

Author: navis
Date: Tue Sep  2 04:41:29 2014
New Revision: 1621912

URL: http://svn.apache.org/r1621912
Log:
HIVE-5799 : session/operation timeout for hiveserver2 (Navis, reviewed by Brock Noland, Lars Francke and Lefty Leverenz)

Added:
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2SessionTimeout.java
Modified:
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/trunk/common/src/java/org/apache/hadoop/hive/conf/Validator.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
    hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
    hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
    hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
    hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/AutoProgressor.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Heartbeater.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsAggregator.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsPublisher.java
    hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/Initiator.java
    hive/trunk/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestInitiator.java
    hive/trunk/ql/src/test/results/clientpositive/show_conf.q.out
    hive/trunk/service/src/java/org/apache/hadoop/hive/service/HiveServer.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/CLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/OperationState.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/Operation.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/operation/OperationManager.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSession.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionBase.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImpl.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/session/SessionManager.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java
    hive/trunk/service/src/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java
    hive/trunk/service/src/test/org/apache/hive/service/cli/CLIServiceTest.java

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=1621912&r1=1621911&r2=1621912&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 Sep  2 04:41:29 2014
@@ -31,6 +31,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -45,6 +46,7 @@ import org.apache.hadoop.hive.conf.Valid
 import org.apache.hadoop.hive.conf.Validator.RangeValidator;
 import org.apache.hadoop.hive.conf.Validator.RatioValidator;
 import org.apache.hadoop.hive.conf.Validator.StringSet;
+import org.apache.hadoop.hive.conf.Validator.TimeValidator;
 import org.apache.hadoop.hive.shims.ShimLoader;
 import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -353,9 +355,11 @@ public class HiveConf extends Configurat
     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,
+    METASTORE_CLIENT_CONNECT_RETRY_DELAY("hive.metastore.client.connect.retry.delay", "1s",
+        new TimeValidator(TimeUnit.SECONDS),
         "Number of seconds for the client to wait between consecutive connection attempts"),
-    METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", 600,
+    METASTORE_CLIENT_SOCKET_TIMEOUT("hive.metastore.client.socket.timeout", "600s",
+        new TimeValidator(TimeUnit.SECONDS),
         "MetaStore Client socket timeout in seconds"),
     METASTOREPWD("javax.jdo.option.ConnectionPassword", "mine",
         "password to use against metastore database"),
@@ -368,9 +372,9 @@ public class HiveConf extends Configurat
         "JDBC connect string for a JDBC metastore"),
 
     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"),
+        "The number of times to retry a HMSHandler call if there were a connection error."),
+    HMSHANDLERINTERVAL("hive.hmshandler.retry.interval", "1000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS), "The time between HMSHandler retry attempts on failure."),
     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" +
@@ -465,10 +469,12 @@ public class HiveConf extends Configurat
         "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)."),
-    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_EVENT_CLEAN_FREQ("hive.metastore.event.clean.freq", "0s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Frequency at which timer task runs to purge expired events in metastore."),
+    METASTORE_EVENT_EXPIRY_DURATION("hive.metastore.event.expiry.duration", "0s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Duration after which events expire from events table"),
     METASTORE_EXECUTE_SET_UGI("hive.metastore.execute.setugi", true,
         "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 \n" +
@@ -580,8 +586,9 @@ public class HiveConf extends Configurat
     HIVE_CURRENT_DATABASE("hive.current.database", "", "Database name used by current session. Internal usage only.", true),
 
     // for hive script operator
-    HIVES_AUTO_PROGRESS_TIMEOUT("hive.auto.progress.timeout", 0,
-        "How long to run autoprogressor for the script/UDTF operators (in seconds).\n" +
+    HIVES_AUTO_PROGRESS_TIMEOUT("hive.auto.progress.timeout", "0s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "How long to run autoprogressor for the script/UDTF operators.\n" +
         "Set to 0 for forever."),
     HIVETABLENAME("hive.table.name", "", ""),
     HIVEPARTITIONNAME("hive.partition.name", "", ""),
@@ -690,10 +697,9 @@ public class HiveConf extends Configurat
         "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, SequenceFile, RCfile and ORC. Users can explicitly override it by CREATE TABLE ... STORED AS [FORMAT]"),
+        "Default file format for CREATE TABLE statement. Users can explicitly override it by CREATE TABLE ... STORED AS [FORMAT]"),
     HIVEQUERYRESULTFILEFORMAT("hive.query.result.fileformat", "TextFile", new StringSet("TextFile", "SequenceFile", "RCfile"),
-        "Default file format for storing result of the query. Allows TextFile, SequenceFile and RCfile"),
+        "Default file format for storing result of the query."),
     HIVECHECKFILEFORMAT("hive.fileformat.check", true, "Whether to check file format or not when loading data files"),
 
     // default serde for rcfile
@@ -720,8 +726,9 @@ public class HiveConf extends Configurat
         "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" +
+    HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL("hive.querylog.plan.progress.interval", "60000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "The interval to wait between logging the plan's progress.\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" +
@@ -841,8 +848,7 @@ public class HiveConf extends Configurat
     HIVE_ORC_ENCODING_STRATEGY("hive.exec.orc.encoding.strategy", "SPEED", new StringSet("SPEED", "COMPRESSION"),
         "Define the encoding strategy to use while writing data. Changing this will\n" +
         "only affect the light weight encoding for integers. This flag will not\n" +
-        "change the compression level of higher level compression codec (like ZLIB).\n" +
-        "Possible options are SPEED and COMPRESSION."),
+        "change the compression level of higher level compression codec (like ZLIB)."),
 
     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" +
@@ -1101,16 +1107,17 @@ public class HiveConf extends Configurat
         "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_JDBC_TIMEOUT("hive.stats.jdbc.timeout", "30s", new TimeValidator(TimeUnit.SECONDS),
+        "Timeout value 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 " +
-        "baseWindow * failures baseWindow * (failure  1) * (random number between [0.0,1.0])."),
+    HIVE_STATS_RETRIES_WAIT("hive.stats.retries.wait", "3000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "The base waiting window before the next retry. The actual wait time is calculated by " +
+        "baseWindow * failures baseWindow * (failure + 1) * (random number between [0.0,1.0])."),
     HIVE_STATS_COLLECT_RAWDATASIZE("hive.stats.collect.rawdatasize", true,
         "should the raw data size be collected when analyzing tables"),
     CLIENT_STATS_COUNTERS("hive.client.stats.counters", "",
@@ -1220,8 +1227,9 @@ public class HiveConf extends Configurat
         "The number of times you want to try to get all the locks"),
     HIVE_UNLOCK_NUMRETRIES("hive.unlock.numretries", 10,
         "The number of times you want to retry to do one unlock"),
-    HIVE_LOCK_SLEEP_BETWEEN_RETRIES("hive.lock.sleep.between.retries", 60,
-        "The sleep time (in seconds) between various retries"),
+    HIVE_LOCK_SLEEP_BETWEEN_RETRIES("hive.lock.sleep.between.retries", "60s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "The sleep time between various retries"),
     HIVE_LOCK_MAPRED_ONLY("hive.lock.mapred.only.operation", false,
         "This param is to control whether or not only do lock on queries\n" +
         "that need to execute at least one mapred job."),
@@ -1241,8 +1249,8 @@ public class HiveConf extends Configurat
     // Transactions
     HIVE_TXN_MANAGER("hive.txn.manager",
         "org.apache.hadoop.hive.ql.lockmgr.DummyTxnManager", ""),
-    HIVE_TXN_TIMEOUT("hive.txn.timeout", 300,
-        "time after which transactions are declared aborted if the client has not sent a heartbeat, in seconds."),
+    HIVE_TXN_TIMEOUT("hive.txn.timeout", "300s", new TimeValidator(TimeUnit.SECONDS),
+        "time after which transactions are declared aborted if the client has not sent a heartbeat."),
 
     HIVE_TXN_MAX_OPEN_BATCH("hive.txn.max.open.batch", 1000,
         "Maximum number of transactions that can be fetched in one call to open_txns().\n" +
@@ -1256,12 +1264,14 @@ public class HiveConf extends Configurat
     HIVE_COMPACTOR_WORKER_THREADS("hive.compactor.worker.threads", 0,
         "Number of compactor worker threads to run on this metastore instance."),
 
-    HIVE_COMPACTOR_WORKER_TIMEOUT("hive.compactor.worker.timeout", 86400L,
-        "Time in seconds, before a given compaction in working state is declared a failure\n" +
+    HIVE_COMPACTOR_WORKER_TIMEOUT("hive.compactor.worker.timeout", "86400s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Time before a given compaction in working state is declared a failure\n" +
         "and returned to the initiated state."),
 
-    HIVE_COMPACTOR_CHECK_INTERVAL("hive.compactor.check.interval", 300L,
-        "Time in seconds between checks to see if any partitions need compacted.\n" +
+    HIVE_COMPACTOR_CHECK_INTERVAL("hive.compactor.check.interval", "300s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Time between checks to see if any partitions need compacted.\n" +
         "This should be kept high because each check for compaction requires many calls against the NameNode."),
 
     HIVE_COMPACTOR_DELTA_NUM_THRESHOLD("hive.compactor.delta.num.threshold", 10,
@@ -1298,7 +1308,7 @@ public class HiveConf extends Configurat
         "Currently the query should be single sourced not having any subquery and should not have\n" +
         "any aggregations or distincts (which incurs RS), lateral views and joins.\n" +
         "1. minimal : SELECT STAR, FILTER on partition columns, LIMIT only\n" +
-        "2. more    : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)\n"
+        "2. more    : SELECT, FILTER, LIMIT only (support TABLESAMPLE and virtual columns)"
     ),
     HIVEFETCHTASKCONVERSIONTHRESHOLD("hive.fetch.task.conversion.threshold", 1073741824L,
         "Input threshold for applying hive.fetch.task.conversion. If target table is native, input length\n" +
@@ -1470,12 +1480,12 @@ public class HiveConf extends Configurat
         "table. From 0.12 onwards, they are displayed separately. This flag will let you\n" +
         "get old behavior, if desired. See, test-case in patch for HIVE-6689."),
 
-    HIVE_SERVER2_MAX_START_ATTEMPTS("hive.server2.max.start.attempts", 30L, new RangeValidator(0L, Long.MAX_VALUE),
+    HIVE_SERVER2_MAX_START_ATTEMPTS("hive.server2.max.start.attempts", 30L, new RangeValidator(0L, null),
         "This number of times HiveServer2 will attempt to start before exiting, sleeping 60 seconds between retries. \n" +
         "The default of 30 will keep trying for 30 minutes."),
 
     HIVE_SERVER2_TRANSPORT_MODE("hive.server2.transport.mode", "binary", new StringSet("binary", "http"),
-        "Server transport mode. \"binary\" or \"http\""),
+        "Transport mode of HiveServer2."),
 
     // http (over thrift) transport settings
     HIVE_SERVER2_THRIFT_HTTP_PORT("hive.server2.thrift.http.port", 10001,
@@ -1486,11 +1496,13 @@ public class HiveConf extends Configurat
         "Minimum number of worker threads when in HTTP mode."),
     HIVE_SERVER2_THRIFT_HTTP_MAX_WORKER_THREADS("hive.server2.thrift.http.max.worker.threads", 500,
         "Maximum number of worker threads when in HTTP mode."),
-    HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME("hive.server2.thrift.http.max.idle.time", 1800000,
-        "Maximum idle time in milliseconds for a connection on the server when in HTTP mode."),
-    HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME("hive.server2.thrift.http.worker.keepalive.time", 60,
-        "Keepalive time (in seconds) for an idle http worker thread. When number of workers > min workers, " +
-        "excess threads are killed after this time interval."),
+    HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME("hive.server2.thrift.http.max.idle.time", "1800s",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "Maximum idle time for a connection on the server when in HTTP mode."),
+    HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME("hive.server2.thrift.http.worker.keepalive.time", "60s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Keepalive time for an idle http worker thread. When the number of workers exceeds min workers, " +
+        "excessive threads are killed after this time interval."),
 
     // binary transport settings
     HIVE_SERVER2_THRIFT_PORT("hive.server2.thrift.port", 10000,
@@ -1513,23 +1525,26 @@ public class HiveConf extends Configurat
         "Minimum number of Thrift worker threads"),
     HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS("hive.server2.thrift.max.worker.threads", 500,
         "Maximum number of Thrift worker threads"),
-    HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME("hive.server2.thrift.worker.keepalive.time", 60,
-        "Keepalive time (in seconds) for an idle worker thread. When number of workers > min workers, " +
-        "excess threads are killed after this time interval."),
+    HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME("hive.server2.thrift.worker.keepalive.time", "60s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Keepalive time (in seconds) for an idle worker thread. When the number of workers exceeds min workers, " +
+        "excessive threads are killed after this time interval."),
     // Configuration for async thread pool in SessionManager
     HIVE_SERVER2_ASYNC_EXEC_THREADS("hive.server2.async.exec.threads", 100,
         "Number of threads in the async thread pool for HiveServer2"),
-    HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", 10,
-        "Time (in seconds) for which HiveServer2 shutdown will wait for async"),
+    HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT("hive.server2.async.exec.shutdown.timeout", "10s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Maximum time for which HiveServer2 shutdown will wait for async"),
     HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE("hive.server2.async.exec.wait.queue.size", 100,
         "Size of the wait queue for async thread pool in HiveServer2.\n" +
         "After hitting this limit, the async thread pool will reject new requests."),
-    HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME("hive.server2.async.exec.keepalive.time", 10,
-        "Time (in seconds) that an idle HiveServer2 async thread (from the thread pool) will wait\n" +
-        "for a new task to arrive before terminating"),
-    HIVE_SERVER2_LONG_POLLING_TIMEOUT("hive.server2.long.polling.timeout", 5000L,
-        "Time in milliseconds that HiveServer2 will wait,\n" +
-        "before responding to asynchronous calls that use long polling"),
+    HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME("hive.server2.async.exec.keepalive.time", "10s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Time that an idle HiveServer2 async thread (from the thread pool) will wait for a new task\n" +
+        "to arrive before terminating"),
+    HIVE_SERVER2_LONG_POLLING_TIMEOUT("hive.server2.long.polling.timeout", "5000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "Time that HiveServer2 will wait before responding to asynchronous calls that use long polling"),
 
     // HiveServer2 auth configuration
     HIVE_SERVER2_AUTHENTICATION("hive.server2.authentication", "NONE",
@@ -1594,6 +1609,18 @@ public class HiveConf extends Configurat
     HIVE_SECURITY_COMMAND_WHITELIST("hive.security.command.whitelist", "set,reset,dfs,add,list,delete,compile",
         "Comma separated list of non-SQL Hive commands users are authorized to execute"),
 
+    HIVE_SERVER2_SESSION_CHECK_INTERVAL("hive.server2.session.check.interval", "0ms",
+        new TimeValidator(TimeUnit.MILLISECONDS, 3000l, true, null, false),
+        "The check interval for session/operation timeout, which can be disabled by setting to zero or negative value."),
+    HIVE_SERVER2_IDLE_SESSION_TIMEOUT("hive.server2.idle.session.timeout", "0ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "Session will be closed when it's not accessed for this duration, which can be disabled by setting to zero or negative value."),
+    HIVE_SERVER2_IDLE_OPERATION_TIMEOUT("hive.server2.idle.operation.timeout", "0ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "Operation will be closed when it's not accessed for this duration of time, which can be disabled by setting to zero value.\n" +
+        "  With positive value, it's checked for operations in terminal state only (FINISHED, CANCELED, CLOSED, ERROR).\n" +
+        "  With negative value, it's checked for all of the operations regardless of state."),
+
     HIVE_CONF_RESTRICTED_LIST("hive.conf.restricted.list",
         "hive.security.authenticator.manager,hive.security.authorization.manager,hive.users.in.admin.role",
         "Comma separated list of configuration options which are immutable at runtime"),
@@ -1651,8 +1678,9 @@ public class HiveConf extends Configurat
         "Enable list bucketing optimizer. Default value is false so that we disable it by default."),
 
     // Allow TCP Keep alive socket option for for HiveServer or a maximum timeout for the socket.
-    SERVER_READ_SOCKET_TIMEOUT("hive.server.read.socket.timeout", 10,
-        "Timeout for the HiveServer to close the connection if no response from the client in N seconds, defaults to 10 seconds."),
+    SERVER_READ_SOCKET_TIMEOUT("hive.server.read.socket.timeout", "10s",
+        new TimeValidator(TimeUnit.SECONDS),
+        "Timeout for the HiveServer to close the connection if no response from the client. By default, 10 seconds."),
     SERVER_TCP_KEEP_ALIVE("hive.server.tcp.keepalive", true,
         "Whether to enable TCP keepalive for the Hive Server. Keepalive will prevent accumulation of half-open connections."),
 
@@ -1711,8 +1739,9 @@ public class HiveConf extends Configurat
         "turning on Tez for HiveServer2. The user could potentially want to run queries\n" +
         "over Tez without the pool of sessions."),
 
-    HIVE_QUOTEDID_SUPPORT("hive.support.quoted.identifiers", "column", new PatternSet("none", "column"),
-        "Whether to use quoted identifier. 'none' ot 'column' can be used. \n" +
+    HIVE_QUOTEDID_SUPPORT("hive.support.quoted.identifiers", "column",
+        new StringSet("none", "column"),
+        "Whether to use quoted identifier. 'none' or 'column' can be used. \n" +
         "  none: default(past) behavior. Implies only alphaNumeric and underscore are valid characters in identifiers.\n" +
         "  column: implies column names can contain any character."
     ),
@@ -1732,8 +1761,9 @@ public class HiveConf extends Configurat
 
     HIVE_CHECK_CROSS_PRODUCT("hive.exec.check.crossproducts", true,
         "Check if a plan contains a Cross Product. If there is one, output a warning to the Session's console."),
-    HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL("hive.localize.resource.wait.interval", 5000L,
-        "Time in milliseconds to wait for another thread to localize the same resource for hive-tez."),
+    HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL("hive.localize.resource.wait.interval", "5000ms",
+        new TimeValidator(TimeUnit.MILLISECONDS),
+        "Time to wait for another thread to localize the same resource for hive-tez."),
     HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS("hive.localize.resource.num.wait.attempts", 5,
         "The number of attempts waiting for localizing a resource in hive-tez."),
     TEZ_AUTO_REDUCER_PARALLELISM("hive.tez.auto.reducer.parallelism", false,
@@ -1843,11 +1873,29 @@ public class HiveConf extends Configurat
       return validator == null ? null : validator.validate(value);
     }
 
+    public String validatorDescription() {
+      return validator == null ? null : validator.toDescription();
+    }
+
     public String typeString() {
-      return valType.typeString();
+      String type = valType.typeString();
+      if (valType == VarType.STRING && validator != null) {
+        if (validator instanceof TimeValidator) {
+          type += "(TIME)";
+        }
+      }
+      return type;
+    }
+
+    public String getRawDescription() {
+      return description;
     }
 
     public String getDescription() {
+      String validator = validatorDescription();
+      if (validator != null) {
+        return validator + ".\n" + description;
+      }
       return description;
     }
 
@@ -1983,6 +2031,82 @@ public class HiveConf extends Configurat
     setIntVar(this, var, val);
   }
 
+  public static long getTimeVar(Configuration conf, ConfVars var, TimeUnit outUnit) {
+    return toTime(getVar(conf, var), getDefaultTimeUnit(var), outUnit);
+  }
+
+  public static void setTimeVar(Configuration conf, ConfVars var, long time, TimeUnit timeunit) {
+    assert (var.valClass == String.class) : var.varname;
+    conf.set(var.varname, time + stringFor(timeunit));
+  }
+
+  public long getTimeVar(ConfVars var, TimeUnit outUnit) {
+    return getTimeVar(this, var, outUnit);
+  }
+
+  public void setTimeVar(ConfVars var, long time, TimeUnit outUnit) {
+    setTimeVar(this, var, time, outUnit);
+  }
+
+  private static TimeUnit getDefaultTimeUnit(ConfVars var) {
+    TimeUnit inputUnit = null;
+    if (var.validator instanceof TimeValidator) {
+      inputUnit = ((TimeValidator)var.validator).getTimeUnit();
+    }
+    return inputUnit;
+  }
+
+  public static long toTime(String value, TimeUnit inputUnit, TimeUnit outUnit) {
+    String[] parsed = parseTime(value.trim());
+    return outUnit.convert(Long.valueOf(parsed[0].trim().trim()), unitFor(parsed[1].trim(), inputUnit));
+  }
+
+  private static String[] parseTime(String value) {
+    char[] chars = value.toCharArray();
+    int i = 0;
+    for (; i < chars.length && (chars[i] == '-' || Character.isDigit(chars[i])); i++) {
+    }
+    return new String[] {value.substring(0, i), value.substring(i)};
+  }
+
+  public static TimeUnit unitFor(String unit, TimeUnit defaultUnit) {
+    unit = unit.trim().toLowerCase();
+    if (unit.isEmpty()) {
+      if (defaultUnit == null) {
+        throw new IllegalArgumentException("Time unit is not specified");
+      }
+      return defaultUnit;
+    } else if (unit.equals("d") || unit.startsWith("day")) {
+      return TimeUnit.DAYS;
+    } else if (unit.equals("h") || unit.startsWith("hour")) {
+      return TimeUnit.HOURS;
+    } else if (unit.equals("m") || unit.startsWith("min")) {
+      return TimeUnit.MINUTES;
+    } else if (unit.equals("s") || unit.startsWith("sec")) {
+      return TimeUnit.SECONDS;
+    } else if (unit.equals("ms") || unit.startsWith("msec")) {
+      return TimeUnit.MILLISECONDS;
+    } else if (unit.equals("us") || unit.startsWith("usec")) {
+      return TimeUnit.MICROSECONDS;
+    } else if (unit.equals("ns") || unit.startsWith("nsec")) {
+      return TimeUnit.NANOSECONDS;
+    }
+    throw new IllegalArgumentException("Invalid time unit " + unit);
+  }
+
+  public static String stringFor(TimeUnit timeunit) {
+    switch (timeunit) {
+      case DAYS: return "day";
+      case HOURS: return "hour";
+      case MINUTES: return "min";
+      case SECONDS: return "sec";
+      case MILLISECONDS: return "msec";
+      case MICROSECONDS: return "usec";
+      case NANOSECONDS: return "nsec";
+    }
+    throw new IllegalArgumentException("Invalid timeunit " + timeunit);
+  }
+
   public static long getLongVar(Configuration conf, ConfVars var) {
     assert (var.valClass == Long.class) : var.varname;
     return conf.getLong(var.varname, var.defaultLongVal);

Modified: hive/trunk/common/src/java/org/apache/hadoop/hive/conf/Validator.java
URL: http://svn.apache.org/viewvc/hive/trunk/common/src/java/org/apache/hadoop/hive/conf/Validator.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/common/src/java/org/apache/hadoop/hive/conf/Validator.java (original)
+++ hive/trunk/common/src/java/org/apache/hadoop/hive/conf/Validator.java Tue Sep  2 04:41:29 2014
@@ -22,6 +22,7 @@ import java.util.ArrayList;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Pattern;
 
 /**
@@ -31,57 +32,85 @@ public interface Validator {
 
   String validate(String value);
 
-  static class StringSet implements Validator {
+  String toDescription();
 
+  class StringSet implements Validator {
+
+    private final boolean caseSensitive;
     private final Set<String> expected = new LinkedHashSet<String>();
 
     public StringSet(String... values) {
+      this(false, values);
+    }
+
+    public StringSet(boolean caseSensitive, String... values) {
+      this.caseSensitive = caseSensitive;
       for (String value : values) {
-        expected.add(value.toLowerCase());
+        expected.add(caseSensitive ? value : value.toLowerCase());
       }
     }
 
     @Override
     public String validate(String value) {
-      if (value == null || !expected.contains(value.toLowerCase())) {
+      if (value == null || !expected.contains(caseSensitive ? value : value.toLowerCase())) {
         return "Invalid value.. expects one of " + expected;
       }
       return null;
     }
+
+    @Override
+    public String toDescription() {
+      return "Expects one of " + expected;
+    }
   }
 
-  static enum RANGE_TYPE {
+  enum TYPE {
     INT {
       @Override
       protected boolean inRange(String value, Object lower, Object upper) {
         int ivalue = Integer.parseInt(value);
-        return (Integer)lower <= ivalue && ivalue <= (Integer)upper;
+        if (lower != null && ivalue < (Integer)lower) {
+          return false;
+        }
+        if (upper != null && ivalue > (Integer)upper) {
+          return false;
+        }
+        return true;
       }
     },
     LONG {
       @Override
       protected boolean inRange(String value, Object lower, Object upper) {
         long lvalue = Long.parseLong(value);
-        return (Long)lower <= lvalue && lvalue <= (Long)upper;
+        if (lower != null && lvalue < (Long)lower) {
+          return false;
+        }
+        if (upper != null && lvalue > (Long)upper) {
+          return false;
+        }
+        return true;
       }
     },
     FLOAT {
       @Override
       protected boolean inRange(String value, Object lower, Object upper) {
         float fvalue = Float.parseFloat(value);
-        return (Float)lower <= fvalue && fvalue <= (Float)upper;
+        if (lower != null && fvalue < (Float)lower) {
+          return false;
+        }
+        if (upper != null && fvalue > (Float)upper) {
+          return false;
+        }
+        return true;
       }
     };
 
-    public static RANGE_TYPE valueOf(Object lower, Object upper) {
-      if (lower instanceof Integer && upper instanceof Integer) {
-        assert (Integer)lower < (Integer)upper;
+    public static TYPE valueOf(Object lower, Object upper) {
+      if (lower instanceof Integer || upper instanceof Integer) {
         return INT;
-      } else if (lower instanceof Long && upper instanceof Long) {
-        assert (Long)lower < (Long)upper;
+      } else if (lower instanceof Long || upper instanceof Long) {
         return LONG;
-      } else if (lower instanceof Float && upper instanceof Float) {
-        assert (Float)lower < (Float)upper;
+      } else if (lower instanceof Float || upper instanceof Float) {
         return FLOAT;
       }
       throw new IllegalArgumentException("invalid range from " + lower + " to " + upper);
@@ -90,15 +119,15 @@ public interface Validator {
     protected abstract boolean inRange(String value, Object lower, Object upper);
   }
 
-  static class RangeValidator implements Validator {
+  class RangeValidator implements Validator {
 
-    private final RANGE_TYPE type;
+    private final TYPE type;
     private final Object lower, upper;
 
     public RangeValidator(Object lower, Object upper) {
       this.lower = lower;
       this.upper = upper;
-      this.type = RANGE_TYPE.valueOf(lower, upper);
+      this.type = TYPE.valueOf(lower, upper);
     }
 
     @Override
@@ -115,9 +144,23 @@ public interface Validator {
       }
       return null;
     }
+
+    @Override
+    public String toDescription() {
+      if (lower == null && upper == null) {
+        return null;
+      }
+      if (lower != null && upper != null) {
+        return "Expects value between " + lower + " and " + upper;
+      }
+      if (lower != null) {
+        return "Expects value bigger than " + lower;
+      }
+      return "Expects value smaller than " + upper;
+    }
   }
 
-  static class PatternSet implements Validator {
+  class PatternSet implements Validator {
 
     private final List<Pattern> expected = new ArrayList<Pattern>();
 
@@ -139,9 +182,14 @@ public interface Validator {
       }
       return "Invalid value.. expects one of patterns " + expected;
     }
+
+    @Override
+    public String toDescription() {
+      return "Expects one of the pattern in " + expected;
+    }
   }
 
-  static class RatioValidator implements Validator {
+  class RatioValidator implements Validator {
 
     @Override
     public String validate(String value) {
@@ -155,5 +203,77 @@ public interface Validator {
       }
       return null;
     }
+
+    @Override
+    public String toDescription() {
+      return "Expects value between 0.0f and 1.0f";
+    }
+  }
+
+  class TimeValidator implements Validator {
+
+    private final TimeUnit timeUnit;
+
+    private final Long min;
+    private final boolean minInclusive;
+
+    private final Long max;
+    private final boolean maxInclusive;
+
+    public TimeValidator(TimeUnit timeUnit) {
+      this(timeUnit, null, false, null, false);
+    }
+
+    public TimeValidator(TimeUnit timeUnit,
+        Long min, boolean minInclusive, Long max, boolean maxInclusive) {
+      this.timeUnit = timeUnit;
+      this.min = min;
+      this.minInclusive = minInclusive;
+      this.max = max;
+      this.maxInclusive = maxInclusive;
+    }
+
+    public TimeUnit getTimeUnit() {
+      return timeUnit;
+    }
+
+    @Override
+    public String validate(String value) {
+      try {
+        long time = HiveConf.toTime(value, timeUnit, timeUnit);
+        if (min != null && (minInclusive ? time < min : time <= min)) {
+          return value + " is smaller than " + timeString(min);
+        }
+        if (max != null && (maxInclusive ? time > max : time >= max)) {
+          return value + " is bigger than " + timeString(max);
+        }
+      } catch (Exception e) {
+        return e.toString();
+      }
+      return null;
+    }
+
+    public String toDescription() {
+      String description =
+          "Expects a time value with unit " +
+          "(d/day, h/hour, m/min, s/sec, ms/msec, us/usec, ns/nsec)" +
+          ", which is " + HiveConf.stringFor(timeUnit) + " if not specified";
+      if (min != null && max != null) {
+        description += ".\nThe time should be in between " +
+            timeString(min) + (minInclusive ? " (inclusive)" : " (exclusive)") + " and " +
+            timeString(max) + (maxInclusive ? " (inclusive)" : " (exclusive)");
+      } else if (min != null) {
+        description += ".\nThe time should be bigger than " +
+            (minInclusive ? "or equal to " : "") + timeString(min);
+      } else if (max != null) {
+        description += ".\nThe time should be smaller than " +
+            (maxInclusive ? "or equal to " : "") + timeString(max);
+      }
+      return description;
+    }
+
+    private String timeString(long time) {
+      return time + " " + HiveConf.stringFor(timeUnit);
+    }
   }
 }

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestPermsGrp.java Tue Sep  2 04:41:29 2014
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.TestCase;
 
@@ -91,13 +92,11 @@ public class TestPermsGrp extends TestCa
     hcatConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://127.0.0.1:" + msPort);
     hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
     hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3);
-    hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 120);
 
     hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname, HCatSemanticAnalyzer.class.getName());
     hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");
     hcatConf.set(HiveConf.ConfVars.POSTEXECHOOKS.varname, "");
-    hcatConf.set(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
-    hcatConf.set(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT.varname, "60");
+    hcatConf.setTimeVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 60, TimeUnit.SECONDS);
     hcatConf.setBoolVar(HiveConf.ConfVars.HIVE_SUPPORT_CONCURRENCY, false);
     clientWH = new Warehouse(hcatConf);
     msc = new HiveMetaStoreClient(hcatConf, null);

Modified: hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java
URL: http://svn.apache.org/viewvc/hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java (original)
+++ hive/trunk/hcatalog/core/src/test/java/org/apache/hive/hcatalog/mapreduce/TestHCatPartitionPublish.java Tue Sep  2 04:41:29 2014
@@ -25,6 +25,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 
@@ -116,7 +117,7 @@ public class TestHCatPartitionPublish {
         + msPort);
     hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
     hcatConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES, 3);
-    hcatConf.setIntVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 120);
+    hcatConf.setTimeVar(HiveConf.ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, 120, TimeUnit.SECONDS);
     hcatConf.set(HiveConf.ConfVars.SEMANTIC_ANALYZER_HOOK.varname,
         HCatSemanticAnalyzer.class.getName());
     hcatConf.set(HiveConf.ConfVars.PREEXECHOOKS.varname, "");

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestMetaStoreAuthorization.java Tue Sep  2 04:41:29 2014
@@ -20,6 +20,7 @@ package org.apache.hadoop.hive.metastore
 
 import java.io.IOException;
 import java.net.ServerSocket;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.TestCase;
 
@@ -45,7 +46,7 @@ public class TestMetaStoreAuthorization 
         "true");
     conf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
     conf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
-    conf.setIntVar(ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, 60);
+    conf.setTimeVar(ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, 60, TimeUnit.SECONDS);
   }
 
   public void testIsWritable() throws Exception {

Modified: hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java (original)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hadoop/hive/metastore/TestRetryingHMSHandler.java Tue Sep  2 04:41:29 2014
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.metastore
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import junit.framework.Assert;
 import junit.framework.TestCase;
@@ -58,7 +59,7 @@ public class TestRetryingHMSHandler exte
     hiveConf.setVar(HiveConf.ConfVars.METASTOREURIS, "thrift://localhost:" + port);
     hiveConf.setIntVar(HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES, 3);
     hiveConf.setIntVar(HiveConf.ConfVars.HMSHANDLERATTEMPTS, 2);
-    hiveConf.setIntVar(HiveConf.ConfVars.HMSHANDLERINTERVAL, 0);
+    hiveConf.setTimeVar(HiveConf.ConfVars.HMSHANDLERINTERVAL, 0, TimeUnit.MILLISECONDS);
     hiveConf.setBoolVar(HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF, false);
     msc = new HiveMetaStoreClient(hiveConf, null);
   }

Added: hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2SessionTimeout.java
URL: http://svn.apache.org/viewvc/hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2SessionTimeout.java?rev=1621912&view=auto
==============================================================================
--- hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2SessionTimeout.java (added)
+++ hive/trunk/itests/hive-unit/src/test/java/org/apache/hive/jdbc/miniHS2/TestHiveServer2SessionTimeout.java Tue Sep  2 04:41:29 2014
@@ -0,0 +1,77 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hive.jdbc.miniHS2;
+
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hive.service.cli.CLIServiceClient;
+import org.apache.hive.service.cli.HiveSQLException;
+import org.apache.hive.service.cli.OperationHandle;
+import org.apache.hive.service.cli.SessionHandle;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestHiveServer2SessionTimeout {
+
+  private static MiniHS2 miniHS2 = null;
+  private Map<String, String> confOverlay;
+
+  @BeforeClass
+  public static void beforeTest() throws Exception {
+    miniHS2 = new MiniHS2(new HiveConf());
+  }
+
+  @Before
+  public void setUp() throws Exception {
+    confOverlay = new HashMap<String, String>();
+    confOverlay.put(ConfVars.HIVE_SUPPORT_CONCURRENCY.varname, "false");
+    confOverlay.put(ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL.varname, "3s");
+    confOverlay.put(ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT.varname, "3s");
+    miniHS2.start(confOverlay);
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    miniHS2.stop();
+  }
+
+  @Test
+  public void testConnection() throws Exception {
+    CLIServiceClient serviceClient = miniHS2.getServiceClient();
+    SessionHandle sessHandle = serviceClient.openSession("foo", "bar");
+    OperationHandle handle = serviceClient.executeStatement(sessHandle, "SELECT 1", confOverlay);
+    Thread.sleep(7000);
+    try {
+      serviceClient.closeOperation(handle);
+      fail("Operation should have been closed by timeout!");
+    } catch (HiveSQLException e) {
+      assertTrue(StringUtils.stringifyException(e),
+          e.getMessage().contains("Invalid OperationHandle"));
+    }
+  }
+}

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStore.java Tue Sep  2 04:41:29 2014
@@ -42,6 +42,7 @@ import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
 import java.util.Timer;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.Condition;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -445,7 +446,7 @@ public class HiveMetaStore extends Thrif
         partitionValidationPattern = null;
       }
 
-      long cleanFreq = hiveConf.getLongVar(ConfVars.METASTORE_EVENT_CLEAN_FREQ) * 1000L;
+      long cleanFreq = hiveConf.getTimeVar(ConfVars.METASTORE_EVENT_CLEAN_FREQ, TimeUnit.MILLISECONDS);
       if (cleanFreq > 0) {
         // In default config, there is no timer.
         Timer cleaner = new Timer("Metastore Events Cleaner Thread", true);

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/HiveMetaStoreClient.java Tue Sep  2 04:41:29 2014
@@ -39,6 +39,7 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import javax.security.auth.login.LoginException;
 
@@ -151,7 +152,7 @@ public class HiveMetaStoreClient impleme
 
   // for thrift connects
   private int retries = 5;
-  private int retryDelaySeconds = 0;
+  private long retryDelaySeconds = 0;
 
   static final protected Log LOG = LogFactory.getLog("hive.metastore");
 
@@ -182,7 +183,8 @@ public class HiveMetaStoreClient impleme
 
     // get the number retries
     retries = HiveConf.getIntVar(conf, HiveConf.ConfVars.METASTORETHRIFTCONNECTIONRETRIES);
-    retryDelaySeconds = conf.getIntVar(ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY);
+    retryDelaySeconds = conf.getTimeVar(
+        ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, TimeUnit.SECONDS);
 
     // user wants file store based configuration
     if (conf.getVar(HiveConf.ConfVars.METASTOREURIS) != null) {
@@ -317,13 +319,14 @@ public class HiveMetaStoreClient impleme
     HadoopShims shim = ShimLoader.getHadoopShims();
     boolean useSasl = conf.getBoolVar(ConfVars.METASTORE_USE_THRIFT_SASL);
     boolean useFramedTransport = conf.getBoolVar(ConfVars.METASTORE_USE_THRIFT_FRAMED_TRANSPORT);
-    int clientSocketTimeout = conf.getIntVar(ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT);
+    int clientSocketTimeout = (int) conf.getTimeVar(
+        ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT, TimeUnit.MILLISECONDS);
 
     for (int attempt = 0; !isConnected && attempt < retries; ++attempt) {
       for (URI store : metastoreUris) {
         LOG.info("Trying to connect to metastore with URI " + store);
         try {
-          transport = new TSocket(store.getHost(), store.getPort(), 1000 * clientSocketTimeout);
+          transport = new TSocket(store.getHost(), store.getPort(), clientSocketTimeout);
           if (useSasl) {
             // Wrap thrift connection with SASL for secure connection.
             try {

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/ObjectStore.java Tue Sep  2 04:41:29 2014
@@ -35,6 +35,7 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Properties;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
@@ -160,7 +161,7 @@ public class ObjectStore implements RawS
 
   private static final Map<String, Class> PINCLASSMAP;
   static {
-    Map<String, Class> map = new HashMap();
+    Map<String, Class> map = new HashMap<String, Class>();
     map.put("table", MTable.class);
     map.put("storagedescriptor", MStorageDescriptor.class);
     map.put("serdeinfo", MSerDeInfo.class);
@@ -1081,14 +1082,14 @@ public class ObjectStore implements RawS
     return keys;
   }
 
-  private SerDeInfo converToSerDeInfo(MSerDeInfo ms) throws MetaException {
+  private SerDeInfo convertToSerDeInfo(MSerDeInfo ms) throws MetaException {
     if (ms == null) {
       throw new MetaException("Invalid SerDeInfo object");
     }
     return new SerDeInfo(ms.getName(), ms.getSerializationLib(), convertMap(ms.getParameters()));
   }
 
-  private MSerDeInfo converToMSerDeInfo(SerDeInfo ms) throws MetaException {
+  private MSerDeInfo convertToMSerDeInfo(SerDeInfo ms) throws MetaException {
     if (ms == null) {
       throw new MetaException("Invalid SerDeInfo object");
     }
@@ -1120,7 +1121,7 @@ public class ObjectStore implements RawS
 
     StorageDescriptor sd = new StorageDescriptor(noFS ? null : convertToFieldSchemas(mFieldSchemas),
         msd.getLocation(), msd.getInputFormat(), msd.getOutputFormat(), msd
-        .isCompressed(), msd.getNumBuckets(), converToSerDeInfo(msd
+        .isCompressed(), msd.getNumBuckets(), convertToSerDeInfo(msd
         .getSerDeInfo()), convertList(msd.getBucketCols()), convertToOrders(msd
         .getSortCols()), convertMap(msd.getParameters()));
     SkewedInfo skewedInfo = new SkewedInfo(convertList(msd.getSkewedColNames()),
@@ -1232,7 +1233,7 @@ public class ObjectStore implements RawS
     }
     return new MStorageDescriptor(mcd, sd
         .getLocation(), sd.getInputFormat(), sd.getOutputFormat(), sd
-        .isCompressed(), sd.getNumBuckets(), converToMSerDeInfo(sd
+        .isCompressed(), sd.getNumBuckets(), convertToMSerDeInfo(sd
         .getSerdeInfo()), sd.getBucketCols(),
         convertToMOrders(sd.getSortCols()), sd.getParameters(),
         (null == sd.getSkewedInfo()) ? null
@@ -2395,7 +2396,7 @@ public class ObjectStore implements RawS
    * Makes a JDO query filter string.
    * Makes a JDO query filter string for tables or partitions.
    * @param dbName Database name.
-   * @param table Table. If null, the query returned is over tables in a database.
+   * @param mtable Table. If null, the query returned is over tables in a database.
    *   If not null, the query returned is over partitions in a table.
    * @param filter The filter from which JDOQL filter will be made.
    * @param params Parameters for the filter. Some parameters may be added here.
@@ -6207,7 +6208,7 @@ public class ObjectStore implements RawS
     boolean commited = false;
     long delCnt;
     LOG.debug("Begin executing cleanupEvents");
-    Long expiryTime = HiveConf.getLongVar(getConf(), ConfVars.METASTORE_EVENT_EXPIRY_DURATION) * 1000L;
+    Long expiryTime = HiveConf.getTimeVar(getConf(), ConfVars.METASTORE_EVENT_EXPIRY_DURATION, TimeUnit.MILLISECONDS);
     Long curTime = System.currentTimeMillis();
     try {
       openTransaction();

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingHMSHandler.java Tue Sep  2 04:41:29 2014
@@ -23,6 +23,7 @@ import java.lang.reflect.InvocationTarge
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.commons.logging.Log;
@@ -80,8 +81,8 @@ public class RetryingHMSHandler implemen
     boolean gotNewConnectUrl = false;
     boolean reloadConf = HiveConf.getBoolVar(hiveConf,
         HiveConf.ConfVars.HMSHANDLERFORCERELOADCONF);
-    int retryInterval = HiveConf.getIntVar(hiveConf,
-        HiveConf.ConfVars.HMSHANDLERINTERVAL);
+    long retryInterval = HiveConf.getTimeVar(hiveConf,
+        HiveConf.ConfVars.HMSHANDLERINTERVAL, TimeUnit.MILLISECONDS);
     int retryLimit = HiveConf.getIntVar(hiveConf,
         HiveConf.ConfVars.HMSHANDLERATTEMPTS);
 

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/RetryingMetaStoreClient.java Tue Sep  2 04:41:29 2014
@@ -24,6 +24,7 @@ import java.lang.reflect.InvocationTarge
 import java.lang.reflect.Method;
 import java.lang.reflect.Proxy;
 import java.lang.reflect.UndeclaredThrowableException;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,18 +49,18 @@ public class RetryingMetaStoreClient imp
 
   private final IMetaStoreClient base;
   private final int retryLimit;
-  private final int retryDelaySeconds;
+  private final long retryDelaySeconds;
 
 
 
   protected RetryingMetaStoreClient(HiveConf hiveConf, HiveMetaHookLoader hookLoader,
       Class<? extends IMetaStoreClient> msClientClass) throws MetaException {
     this.retryLimit = hiveConf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES);
-    this.retryDelaySeconds =
-        hiveConf.getIntVar(HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY);
+    this.retryDelaySeconds = hiveConf.getTimeVar(
+        HiveConf.ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY, TimeUnit.SECONDS);
 
     reloginExpiringKeytabUser();
-    this.base = (IMetaStoreClient) MetaStoreUtils.newInstance(msClientClass, new Class[] {
+    this.base = MetaStoreUtils.newInstance(msClientClass, new Class[] {
         HiveConf.class, HiveMetaHookLoader.class}, new Object[] {hiveConf, hookLoader});
   }
 

Modified: hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java (original)
+++ hive/trunk/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java Tue Sep  2 04:41:29 2014
@@ -40,6 +40,7 @@ import javax.sql.DataSource;
 import java.io.IOException;
 import java.sql.*;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 
 /**
  * A handler to answer transaction related calls that come into the metastore
@@ -119,7 +120,7 @@ public class TxnHandler {
       throw new RuntimeException(e);
     }
 
-    timeout = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT) * 1000;
+    timeout = HiveConf.getTimeVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, TimeUnit.MILLISECONDS);
     deadlockCnt = 0;
     buildJumpTable();
   }

Modified: hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java
URL: http://svn.apache.org/viewvc/hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java (original)
+++ hive/trunk/metastore/src/test/org/apache/hadoop/hive/metastore/txn/TestTxnHandler.java Tue Sep  2 04:41:29 2014
@@ -17,7 +17,6 @@
  */
 package org.apache.hadoop.hive.metastore.txn;
 
-import junit.framework.Assert;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -26,11 +25,11 @@ import org.apache.log4j.Level;
 import org.apache.log4j.LogManager;
 import org.junit.After;
 import org.junit.Before;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import static junit.framework.Assert.*;
 
@@ -868,7 +867,7 @@ public class TestTxnHandler {
 
   @Test
   public void testHeartbeatLock() throws Exception {
-    conf.setIntVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1);
+    conf.setTimeVar(HiveConf.ConfVars.HIVE_TXN_TIMEOUT, 1, TimeUnit.SECONDS);
     HeartbeatRequest h = new HeartbeatRequest();
     LockComponent comp = new LockComponent(LockType.EXCLUSIVE, LockLevel.DB, "mydb");
     comp.setTablename("mytable");

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/AutoProgressor.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/AutoProgressor.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/AutoProgressor.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/AutoProgressor.java Tue Sep  2 04:41:29 2014
@@ -44,7 +44,7 @@ public class AutoProgressor {
   // Name of the class to report for
   String logClassName = null;
   int notificationInterval;
-  int timeout;
+  long timeout;
   Reporter reporter;
 
   class ReporterTask extends TimerTask {
@@ -116,7 +116,7 @@ public class AutoProgressor {
    * @param timeout - when the autoprogressor should stop reporting (in ms)
    */
   AutoProgressor(String logClassName, Reporter reporter,
-      int notificationInterval, int timeout) {
+      int notificationInterval, long timeout) {
     this.logClassName = logClassName;
     this.reporter = reporter;
     this.notificationInterval = notificationInterval;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Sep  2 04:41:29 2014
@@ -507,21 +507,19 @@ public class DDLTask extends Task<DDLWor
       throw new HiveException("invalid configuration name " + showConf.getConfName());
     }
     String description = conf.getDescription();
-    String defaltValue = conf.getDefaultValue();
+    String defaultValue = conf.getDefaultValue();
     DataOutputStream output = getOutputStream(showConf.getResFile());
     try {
+      if (defaultValue != null) {
+        output.write(defaultValue.getBytes());
+      }
+      output.write(separator);
+      output.write(conf.typeString().getBytes());
+      output.write(separator);
       if (description != null) {
-        if (defaltValue != null) {
-          output.write(defaltValue.getBytes());
-        }
-        output.write(separator);
-        output.write(conf.typeString().getBytes());
-        output.write(separator);
-        if (description != null) {
-          output.write(description.replaceAll(" *\n *", " ").getBytes());
-        }
-        output.write(terminator);
+        output.write(description.replaceAll(" *\n *", " ").getBytes());
       }
+      output.write(terminator);
     } finally {
       output.close();
     }

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Heartbeater.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Heartbeater.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Heartbeater.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Heartbeater.java Tue Sep  2 04:41:29 2014
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.lockmgr
 import org.apache.hadoop.hive.ql.lockmgr.LockException;
 
 import java.io.IOException;
+import java.util.concurrent.TimeUnit;
 
 /**
  * Class to handle heartbeats for MR and Tez tasks.
@@ -64,7 +65,8 @@ public class Heartbeater {
     if (heartbeatInterval == 0) {
       // Multiply the heartbeat interval by 1000 to convert to milliseconds,
       // but divide by 2 to give us a safety factor.
-      heartbeatInterval = HiveConf.getIntVar(conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT) * 500;
+      heartbeatInterval = HiveConf.getTimeVar(
+          conf, HiveConf.ConfVars.HIVE_TXN_TIMEOUT, TimeUnit.MILLISECONDS) / 2;
       if (heartbeatInterval == 0) {
         LOG.warn(HiveConf.ConfVars.HIVE_TXN_MANAGER.toString() + " not set, heartbeats won't be sent");
         dontHeartbeat = true;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/ScriptOperator.java Tue Sep  2 04:41:29 2014
@@ -31,6 +31,7 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.Timer;
 import java.util.TimerTask;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -157,12 +158,12 @@ public class ScriptOperator extends Oper
   }
 
   /**
-   * Maps a relative pathname to an absolute pathname using the PATH enviroment.
+   * Maps a relative pathname to an absolute pathname using the PATH environment.
    */
   public class PathFinder {
     String pathenv; // a string of pathnames
-    String pathSep; // the path seperator
-    String fileSep; // the file seperator in a directory
+    String pathSep; // the path separator
+    String fileSep; // the file separator in a directory
 
     /**
      * Construct a PathFinder object using the path from the specified system
@@ -284,7 +285,7 @@ public class ScriptOperator extends Oper
 
   @Override
   public void processOp(Object row, int tag) throws HiveException {
-    // initialize the user's process only when you recieve the first row
+    // initialize the user's process only when you receive the first row
     if (firstRow) {
       firstRow = false;
       try {
@@ -358,7 +359,8 @@ public class ScriptOperator extends Oper
             .getBoolVar(hconf, HiveConf.ConfVars.HIVESCRIPTAUTOPROGRESS)) {
           autoProgressor = new AutoProgressor(this.getClass().getName(),
               reporter, Utilities.getDefaultNotificationInterval(hconf),
-              HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT) * 1000);
+              HiveConf.getTimeVar(
+                  hconf, HiveConf.ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT, TimeUnit.MILLISECONDS));
           autoProgressor.go();
         }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/UDTFOperator.java Tue Sep  2 04:41:29 2014
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.exec;
 import java.io.Serializable;
 import java.util.Arrays;
 import java.util.List;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -31,7 +32,6 @@ import org.apache.hadoop.hive.ql.plan.UD
 import org.apache.hadoop.hive.ql.plan.api.OperatorType;
 import org.apache.hadoop.hive.ql.udf.generic.GenericUDTF;
 import org.apache.hadoop.hive.ql.udf.generic.UDTFCollector;
-import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
 import org.apache.hadoop.hive.serde2.objectinspector.StructField;
 import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector;
 
@@ -86,7 +86,8 @@ public class UDTFOperator extends Operat
     if (HiveConf.getBoolVar(hconf, HiveConf.ConfVars.HIVEUDTFAUTOPROGRESS)) {
       autoProgressor = new AutoProgressor(this.getClass().getName(), reporter,
           Utilities.getDefaultNotificationInterval(hconf),
-          HiveConf.getIntVar(hconf, HiveConf.ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT) * 1000);
+          HiveConf.getTimeVar(
+              hconf, HiveConf.ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT, TimeUnit.MILLISECONDS));
       autoProgressor.go();
     }
 

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java Tue Sep  2 04:41:29 2014
@@ -2743,7 +2743,7 @@ public final class Utilities {
    * first time it is caught, or SQLTransientException when the maxRetries has reached.
    */
   public static <T> T executeWithRetry(SQLCommand<T> cmd, PreparedStatement stmt,
-      int baseWindow, int maxRetries)  throws SQLException {
+      long baseWindow, int maxRetries)  throws SQLException {
 
     Random r = new Random();
     T result = null;
@@ -2785,7 +2785,7 @@ public final class Utilities {
    * first time it is caught, or SQLTransientException when the maxRetries has reached.
    */
   public static Connection connectWithRetry(String connectionString,
-      int waitWindow, int maxRetries) throws SQLException {
+      long waitWindow, int maxRetries) throws SQLException {
 
     Random r = new Random();
 
@@ -2827,7 +2827,7 @@ public final class Utilities {
    * first time it is caught, or SQLTransientException when the maxRetries has reached.
    */
   public static PreparedStatement prepareWithRetry(Connection conn, String stmt,
-      int waitWindow, int maxRetries) throws SQLException {
+      long waitWindow, int maxRetries) throws SQLException {
 
     Random r = new Random();
 
@@ -2867,7 +2867,7 @@ public final class Utilities {
    * @param r a random generator.
    * @return number of milliseconds for the next wait time.
    */
-  public static long getRandomWaitTime(int baseWindow, int failures, Random r) {
+  public static long getRandomWaitTime(long baseWindow, int failures, Random r) {
     return (long) (
           baseWindow * failures +     // grace period for the last round of attempt
           baseWindow * (failures + 1) * r.nextDouble()); // expanding time window for each failure
@@ -3376,7 +3376,7 @@ public final class Utilities {
   /**
    * Returns true if a plan is both configured for vectorized execution
    * and vectorization is allowed. The plan may be configured for vectorization
-   * but vectorization dissalowed eg. for FetchOperator execution.
+   * but vectorization disallowed eg. for FetchOperator execution.
    */
   public static boolean isVectorMode(Configuration conf) {
     if (HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_VECTORIZATION_ENABLED) &&

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/HadoopJobExecHelper.java Tue Sep  2 04:41:29 2014
@@ -29,6 +29,7 @@ import java.util.HashMap;
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -219,8 +220,8 @@ public class HadoopJobExecHelper {
     SimpleDateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss,SSS");
     //DecimalFormat longFormatter = new DecimalFormat("###,###");
     long reportTime = System.currentTimeMillis();
-    long maxReportInterval =
-        HiveConf.getLongVar(job, HiveConf.ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL);
+    long maxReportInterval = HiveConf.getTimeVar(
+        job, HiveConf.ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL, TimeUnit.MILLISECONDS);
     boolean fatal = false;
     StringBuilder errMsg = new StringBuilder();
     long pullInterval = HiveConf.getLongVar(job, HiveConf.ConfVars.HIVECOUNTERSPULLINTERVAL);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/exec/tez/DagUtils.java Tue Sep  2 04:41:29 2014
@@ -29,6 +29,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.TimeUnit;
 
 import javax.security.auth.login.LoginException;
 
@@ -95,7 +96,6 @@ import org.apache.tez.dag.api.ProcessorD
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.VertexGroup;
-import org.apache.tez.dag.api.VertexLocationHint;
 import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
 import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager;
 import org.apache.tez.mapreduce.hadoop.InputSplitInfo;
@@ -131,7 +131,7 @@ public class DagUtils {
 
   private void addCredentials(MapWork mapWork, DAG dag) {
     Set<String> paths = mapWork.getPathToAliases().keySet();
-    if (paths != null && !paths.isEmpty()) {
+    if (!paths.isEmpty()) {
       Iterator<URI> pathIterator = Iterators.transform(paths.iterator(), new Function<String, URI>() {
         @Override
         public URI apply(String input) {
@@ -710,7 +710,7 @@ public class DagUtils {
 
   /**
    * Localizes files, archives and jars from a provided array of names.
-   * @param hdfsDirPathStr Destination directoty in HDFS.
+   * @param hdfsDirPathStr Destination directory in HDFS.
    * @param conf Configuration.
    * @param inputOutputJars The file names to localize.
    * @return List<LocalResource> local resources to add to execution
@@ -784,7 +784,7 @@ public class DagUtils {
   }
 
   /**
-   * @param pathStr - the string from which we try to determine the resource base name
+   * @param path - the path from which we try to determine the resource base name
    * @return the name of the resource from a given path string.
    */
   public String getResourceBaseName(Path path) {
@@ -830,9 +830,8 @@ public class DagUtils {
         int waitAttempts =
             conf.getInt(HiveConf.ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.varname,
                 HiveConf.ConfVars.HIVE_LOCALIZE_RESOURCE_NUM_WAIT_ATTEMPTS.defaultIntVal);
-        long sleepInterval =
-            conf.getLong(HiveConf.ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL.varname,
-                HiveConf.ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL.defaultLongVal);
+        long sleepInterval = HiveConf.getTimeVar(
+            conf, HiveConf.ConfVars.HIVE_LOCALIZE_RESOURCE_WAIT_INTERVAL, TimeUnit.MILLISECONDS);
         LOG.info("Number of wait attempts: " + waitAttempts + ". Wait interval: "
             + sleepInterval);
         boolean found = false;
@@ -919,7 +918,6 @@ public class DagUtils {
    * @param work The instance of BaseWork representing the actual work to be performed
    * by this vertex.
    * @param scratchDir HDFS scratch dir for this execution unit.
-   * @param list
    * @param appJarLr Local resource for hive-exec.
    * @param additionalLr
    * @param fileSystem FS corresponding to scratchDir and LocalResources

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/EmbeddedLockManager.java Tue Sep  2 04:41:29 2014
@@ -25,6 +25,7 @@ import org.apache.hadoop.hive.ql.lockmgr
 import org.apache.hadoop.hive.ql.metadata.*;
 
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.ReentrantLock;
 
 /**
@@ -38,7 +39,7 @@ public class EmbeddedLockManager impleme
 
   private HiveLockManagerCtx ctx;
 
-  private int sleepTime = 1000;
+  private long sleepTime = 1000;
   private int numRetriesForLock = 0;
   private int numRetriesForUnLock = 0;
 
@@ -82,12 +83,13 @@ public class EmbeddedLockManager impleme
 
   public void refresh() {
     HiveConf conf = ctx.getConf();
-    sleepTime = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
+    sleepTime = conf.getTimeVar(
+        HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES, TimeUnit.MILLISECONDS);
     numRetriesForLock = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
     numRetriesForUnLock = conf.getIntVar(HiveConf.ConfVars.HIVE_UNLOCK_NUMRETRIES);
   }
 
-  public HiveLock lock(HiveLockObject key, HiveLockMode mode, int numRetriesForLock, int sleepTime)
+  public HiveLock lock(HiveLockObject key, HiveLockMode mode, int numRetriesForLock, long sleepTime)
       throws LockException {
     for (int i = 0; i <= numRetriesForLock; i++) {
       if (i > 0) {
@@ -101,7 +103,7 @@ public class EmbeddedLockManager impleme
     return null;
   }
 
-  private void sleep(int sleepTime) {
+  private void sleep(long sleepTime) {
     try {
       Thread.sleep(sleepTime);
     } catch (InterruptedException e) {
@@ -109,7 +111,7 @@ public class EmbeddedLockManager impleme
     }
   }
 
-  public List<HiveLock> lock(List<HiveLockObj> objs, int numRetriesForLock, int sleepTime)
+  public List<HiveLock> lock(List<HiveLockObj> objs, int numRetriesForLock, long sleepTime)
       throws LockException {
     sortLocks(objs);
     for (int i = 0; i <= numRetriesForLock; i++) {
@@ -132,7 +134,7 @@ public class EmbeddedLockManager impleme
   }
 
   private List<HiveLock> lockPrimitive(List<HiveLockObj> objs, int numRetriesForLock,
-      int sleepTime) throws LockException {
+      long sleepTime) throws LockException {
     List<HiveLock> locks = new ArrayList<HiveLock>();
     for (HiveLockObj obj : objs) {
       HiveLock lock = lockPrimitive(obj.getObj(), obj.getMode());
@@ -164,7 +166,7 @@ public class EmbeddedLockManager impleme
     });
   }
 
-  public void unlock(HiveLock hiveLock, int numRetriesForUnLock, int sleepTime)
+  public void unlock(HiveLock hiveLock, int numRetriesForUnLock, long sleepTime)
       throws LockException {
     String[] paths = hiveLock.getHiveLockObject().getPaths();
     HiveLockObjectData data = hiveLock.getHiveLockObject().getData();
@@ -179,7 +181,7 @@ public class EmbeddedLockManager impleme
     throw new LockException("Failed to release lock " + hiveLock);
   }
 
-  public void releaseLocks(List<HiveLock> hiveLocks, int numRetriesForUnLock, int sleepTime) {
+  public void releaseLocks(List<HiveLock> hiveLocks, int numRetriesForUnLock, long sleepTime) {
     for (HiveLock locked : hiveLocks) {
       try {
         unlock(locked, numRetriesForUnLock, sleepTime);

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/lockmgr/zookeeper/ZooKeeperHiveLockManager.java Tue Sep  2 04:41:29 2014
@@ -37,6 +37,7 @@ import org.apache.zookeeper.ZooKeeper;
 import java.io.IOException;
 import java.net.InetAddress;
 import java.util.*;
+import java.util.concurrent.TimeUnit;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
@@ -53,7 +54,7 @@ public class ZooKeeperHiveLockManager im
   private int sessionTimeout;
   private String quorumServers;
 
-  private int sleepTime;
+  private long sleepTime;
   private int numRetriesForLock;
   private int numRetriesForUnLock;
 
@@ -106,7 +107,8 @@ public class ZooKeeperHiveLockManager im
     sessionTimeout = conf.getIntVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT);
     quorumServers = ZooKeeperHiveLockManager.getQuorumServers(conf);
 
-    sleepTime = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
+    sleepTime = conf.getTimeVar(
+        HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES, TimeUnit.MILLISECONDS);
     numRetriesForLock = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
     numRetriesForUnLock = conf.getIntVar(HiveConf.ConfVars.HIVE_UNLOCK_NUMRETRIES);
 
@@ -132,7 +134,8 @@ public class ZooKeeperHiveLockManager im
   @Override
   public void refresh() {
     HiveConf conf = ctx.getConf();
-    sleepTime = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES) * 1000;
+    sleepTime = conf.getTimeVar(
+        HiveConf.ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES, TimeUnit.MILLISECONDS);
     numRetriesForLock = conf.getIntVar(HiveConf.ConfVars.HIVE_LOCK_NUMRETRIES);
     numRetriesForUnLock = conf.getIntVar(HiveConf.ConfVars.HIVE_UNLOCK_NUMRETRIES);
   }
@@ -268,7 +271,7 @@ public class ZooKeeperHiveLockManager im
    * @param mode
    *          The mode of the lock
    * @param keepAlive
-   *          Whether the lock is to be persisted after the statement Acuire the
+   *          Whether the lock is to be persisted after the statement Acquire the
    *          lock. Return null if a conflicting lock is present.
    **/
   public ZooKeeperHiveLock lock(HiveLockObject key, HiveLockMode mode,
@@ -515,8 +518,8 @@ public class ZooKeeperHiveLockManager im
     try {
       int sessionTimeout = conf.getIntVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT);
       String quorumServers = getQuorumServers(conf);
-      Watcher dummWatcher = new DummyWatcher();
-      zkpClient = new ZooKeeper(quorumServers, sessionTimeout, dummWatcher);
+      Watcher dummyWatcher = new DummyWatcher();
+      zkpClient = new ZooKeeper(quorumServers, sessionTimeout, dummyWatcher);
       String parent = conf.getVar(HiveConf.ConfVars.HIVE_ZOOKEEPER_NAMESPACE);
       List<HiveLock> locks = getLocks(conf, zkpClient, null, parent, false, false);
       Exception lastExceptionGot = null;

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsAggregator.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsAggregator.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsAggregator.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsAggregator.java Tue Sep  2 04:41:29 2014
@@ -27,6 +27,7 @@ import java.sql.SQLRecoverableException;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -46,7 +47,8 @@ public class JDBCStatsAggregator impleme
   private final Log LOG = LogFactory.getLog(this.getClass().getName());
   private int timeout = 30;
   private final String comment = "Hive stats aggregation: " + this.getClass().getName();
-  private int maxRetries, waitWindow;
+  private int maxRetries;
+  private long waitWindow;
   private final Random r;
 
   public JDBCStatsAggregator() {
@@ -57,11 +59,14 @@ public class JDBCStatsAggregator impleme
   @Override
   public boolean connect(Configuration hiveconf, Task sourceTask) {
     this.hiveconf = hiveconf;
-    timeout = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_JDBC_TIMEOUT);
+    timeout = (int) HiveConf.getTimeVar(
+        hiveconf, HiveConf.ConfVars.HIVE_STATS_JDBC_TIMEOUT, TimeUnit.SECONDS);
     connectionString = HiveConf.getVar(hiveconf, HiveConf.ConfVars.HIVESTATSDBCONNECTIONSTRING);
     String driver = HiveConf.getVar(hiveconf, HiveConf.ConfVars.HIVESTATSJDBCDRIVER);
     maxRetries = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_MAX);
-    waitWindow = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_WAIT);
+    waitWindow = HiveConf.getTimeVar(
+        hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_WAIT, TimeUnit.MILLISECONDS);
+    this.sourceTask = sourceTask;
 
     try {
       Class.forName(driver).newInstance();

Modified: hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsPublisher.java
URL: http://svn.apache.org/viewvc/hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsPublisher.java?rev=1621912&r1=1621911&r2=1621912&view=diff
==============================================================================
--- hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsPublisher.java (original)
+++ hive/trunk/ql/src/java/org/apache/hadoop/hive/ql/stats/jdbc/JDBCStatsPublisher.java Tue Sep  2 04:41:29 2014
@@ -30,6 +30,7 @@ import java.sql.Statement;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -48,7 +49,8 @@ public class JDBCStatsPublisher implemen
   private int timeout; // default timeout in sec. for JDBC connection and statements
   // SQL comment that identifies where the SQL statement comes from
   private final String comment = "Hive stats publishing: " + this.getClass().getName();
-  private int maxRetries, waitWindow;
+  private int maxRetries;
+  private long waitWindow;
   private final Random r;
 
   public JDBCStatsPublisher() {
@@ -59,9 +61,11 @@ public class JDBCStatsPublisher implemen
   public boolean connect(Configuration hiveconf) {
     this.hiveconf = hiveconf;
     maxRetries = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_MAX);
-    waitWindow = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_WAIT);
+    waitWindow = HiveConf.getTimeVar(
+        hiveconf, HiveConf.ConfVars.HIVE_STATS_RETRIES_WAIT, TimeUnit.MILLISECONDS);
     connectionString = HiveConf.getVar(hiveconf, HiveConf.ConfVars.HIVESTATSDBCONNECTIONSTRING);
-    timeout = HiveConf.getIntVar(hiveconf, HiveConf.ConfVars.HIVE_STATS_JDBC_TIMEOUT);
+    timeout = (int) HiveConf.getTimeVar(
+        hiveconf, HiveConf.ConfVars.HIVE_STATS_JDBC_TIMEOUT, TimeUnit.SECONDS);
     String driver = HiveConf.getVar(hiveconf, HiveConf.ConfVars.HIVESTATSJDBCDRIVER);
 
     try {