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

svn commit: r1606275 [1/3] - in /hive/branches/cbo: ./ common/src/java/org/apache/hadoop/hive/conf/ conf/ contrib/src/test/queries/clientnegative/ contrib/src/test/queries/clientpositive/ data/conf/ hbase-handler/src/java/org/apache/hadoop/hive/hbase/ ...

Author: gunther
Date: Sat Jun 28 00:23:54 2014
New Revision: 1606275

URL: http://svn.apache.org/r1606275
Log:
Merge latest trunk into cbo branch (Gunther Hagleitner)

Added:
    hive/branches/cbo/itests/qtest/testconfiguration.properties
      - copied unchanged from r1606272, hive/trunk/itests/qtest/testconfiguration.properties
    hive/branches/cbo/metastore/scripts/upgrade/oracle/019-HIVE-7118.oracle.sql
      - copied unchanged from r1606272, hive/trunk/metastore/scripts/upgrade/oracle/019-HIVE-7118.oracle.sql
    hive/branches/cbo/ql/src/test/queries/clientpositive/describe_database.q
      - copied unchanged from r1606272, hive/trunk/ql/src/test/queries/clientpositive/describe_database.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/tez_join_hash.q
      - copied unchanged from r1606272, hive/trunk/ql/src/test/queries/clientpositive/tez_join_hash.q
    hive/branches/cbo/ql/src/test/results/clientpositive/describe_database.q.out
      - copied unchanged from r1606272, hive/trunk/ql/src/test/results/clientpositive/describe_database.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out
      - copied unchanged from r1606272, hive/trunk/ql/src/test/results/clientpositive/tez/tez_join_hash.q.out
Modified:
    hive/branches/cbo/   (props changed)
    hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
    hive/branches/cbo/conf/hive-default.xml.template
    hive/branches/cbo/contrib/src/test/queries/clientnegative/case_with_row_sequence.q
    hive/branches/cbo/contrib/src/test/queries/clientpositive/dboutput.q
    hive/branches/cbo/data/conf/hive-site.xml
    hive/branches/cbo/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
    hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats.q
    hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats2.q
    hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats3.q
    hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats2.q.out
    hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats3.q.out
    hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
    hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceOutputFormat.java
    hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
    hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
    hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
    hive/branches/cbo/itests/qtest/pom.xml
    hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
    hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/VerifyOverriddenConfigsHook.java
    hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java
    hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java
    hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql
    hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
    hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
    hive/branches/cbo/pom.xml
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/ReduceSinkOperator.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/Utilities.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/ExecDriver.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapRedTask.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/mr/MapredLocalTask.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/exec/vector/VectorReduceSinkOperator.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/io/RCFile.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/io/RCFileOutputFormat.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/merge/RCFileMergeMapper.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/io/rcfile/truncate/ColumnTruncateMapper.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/plan/DescDatabaseDesc.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/processors/SetProcessor.java
    hive/branches/cbo/ql/src/java/org/apache/hadoop/hive/ql/udf/generic/GenericUDAFCount.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/exec/TestExecDriver.java
    hive/branches/cbo/ql/src/test/org/apache/hadoop/hive/ql/io/TestRCFile.java
    hive/branches/cbo/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dynpart_sort_opt_vectorization.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/dynpart_sort_optimization.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/fetch_aggregation.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/index_bitmap_compression.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/index_compression.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/join25.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/join36.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/join37.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/join_nulls.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/join_nullsafe.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/metadata_export_drop.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/nonmr_fetch.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/orc_analyze.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/overridden_confs.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/quotedid_skew.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/sample10.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt1.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt10.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt11.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt12.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt13.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt14.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt15.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt16.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt17.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt18.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt19.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt2.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt20.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt3.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt4.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt5.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt6.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt7.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt8.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/skewjoinopt9.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/smb_mapjoin_25.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/stats15.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/truncate_table.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/udtf_explode.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vector_decimal_mapjoin.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vectorized_bucketmapjoin1.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vectorized_mapjoin.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vectorized_nested_mapjoin.q
    hive/branches/cbo/ql/src/test/queries/clientpositive/vectorized_parquet.q
    hive/branches/cbo/ql/src/test/results/clientnegative/cachingprintstream.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/compute_stats_long.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/dyn_part3.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/dyn_part_max_per_node.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/index_compact_entry_limit.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/index_compact_size_limit.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/local_mapred_error_cache.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/script_broken_pipe2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/script_broken_pipe3.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/script_error.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/serde_regex2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/set_hiveconf_validation0.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/set_hiveconf_validation1.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/stats_aggregator_error_2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/stats_publisher_error_1.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/stats_publisher_error_2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/udf_assert_true.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/udf_assert_true2.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/udf_reflect_neg.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/udf_test_error.q.out
    hive/branches/cbo/ql/src/test/results/clientnegative/udf_test_error_reduce.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/alter_db_owner.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/authorization_owner_actions_db.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/auto_join25.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/auto_join_without_localtask.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/database_location.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/database_properties.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/index_bitmap_compression.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/index_compression.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/infer_bucket_sort_convert_join.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/join_nullsafe.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/mapjoin_hook.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/overridden_confs.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/sample8.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/stats15.q.out
    hive/branches/cbo/ql/src/test/results/clientpositive/udtf_explode.q.out
    hive/branches/cbo/serde/src/java/org/apache/hadoop/hive/serde2/objectinspector/StandardStructObjectInspector.java

Propchange: hive/branches/cbo/
------------------------------------------------------------------------------
  Merged /hive/trunk:r1605012-1606272

Modified: hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java (original)
+++ hive/branches/cbo/common/src/java/org/apache/hadoop/hive/conf/HiveConf.java Sat Jun 28 00:23:54 2014
@@ -182,6 +182,7 @@ public class HiveConf extends Configurat
     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),
+    SUBMITLOCALTASKVIACHILD("hive.exec.submit.local.task.via.child", true),
     SCRIPTERRORLIMIT("hive.exec.script.maxerrsize", 100000),
     ALLOWPARTIALCONSUMP("hive.exec.script.allow.partial.consumption", false),
     STREAMREPORTERPERFIX("stream.stderr.reporter.prefix", "reporter:"),
@@ -531,6 +532,8 @@ public class HiveConf extends Configurat
     HIVETESTMODEPREFIX("hive.test.mode.prefix", "test_"),
     HIVETESTMODESAMPLEFREQ("hive.test.mode.samplefreq", 32),
     HIVETESTMODENOSAMPLE("hive.test.mode.nosamplelist", ""),
+    HIVETESTMODEDUMMYSTATAGGR("hive.test.dummystats.aggregator", ""), // internal variable
+    HIVETESTMODEDUMMYSTATPUB("hive.test.dummystats.publisher", ""),   // internal variable
 
     HIVEMERGEMAPFILES("hive.merge.mapfiles", true),
     HIVEMERGEMAPREDFILES("hive.merge.mapredfiles", false),
@@ -546,6 +549,11 @@ public class HiveConf extends Configurat
     HIVEUSEEXPLICITRCFILEHEADER("hive.exec.rcfile.use.explicit.header", true),
     HIVEUSERCFILESYNCCACHE("hive.exec.rcfile.use.sync.cache", true),
 
+    HIVE_RCFILE_RECORD_INTERVAL("hive.io.rcfile.record.interval", Integer.MAX_VALUE),
+    HIVE_RCFILE_COLUMN_NUMBER_CONF("hive.io.rcfile.column.number.conf", 0),
+    HIVE_RCFILE_TOLERATE_CORRUPTIONS("hive.io.rcfile.tolerate.corruptions", false),
+    HIVE_RCFILE_RECORD_BUFFER_SIZE("hive.io.rcfile.record.buffer.size", 4194304),   // 4M
+
     // 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
@@ -792,6 +800,7 @@ public class HiveConf extends Configurat
 
     // For HBase storage handler
     HIVE_HBASE_WAL_ENABLED("hive.hbase.wal.enabled", true),
+    HIVE_HBASE_GENERATE_HFILES("hive.hbase.generatehfiles", false),
 
     // For har files
     HIVEARCHIVEENABLED("hive.archive.enabled", false),
@@ -849,6 +858,8 @@ public class HiveConf extends Configurat
 
     HIVE_ERROR_ON_EMPTY_PARTITION("hive.error.on.empty.partition", false),
 
+    HIVE_INDEX_COMPACT_FILE("hive.index.compact.file", ""),           // internal variable
+    HIVE_INDEX_BLOCKFILTER_FILE("hive.index.blockfilter.file", ""),   // internal variable
     HIVE_INDEX_IGNORE_HDFS_LOC("hive.index.compact.file.ignore.hdfs", false),
 
     HIVE_EXIM_URI_SCHEME_WL("hive.exim.uri.scheme.whitelist", "hdfs,pfile"),
@@ -1434,6 +1445,31 @@ public class HiveConf extends Configurat
       setBoolVar(ConfVars.METASTORE_FIXED_DATASTORE, true);
     }
 
+    if (getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) {
+      List<String> trimmed = new ArrayList<String>();
+      for (Map.Entry<String,String> entry : this) {
+        String key = entry.getKey();
+        if (key == null || !key.startsWith("hive.")) {
+          continue;
+        }
+        ConfVars var = HiveConf.getConfVars(key);
+        if (var == null) {
+          var = HiveConf.getConfVars(key.trim());
+          if (var != null) {
+            trimmed.add(key);
+          }
+        }
+        if (var == null) {
+          l4j.warn("HiveConf of name " + key + " does not exist");
+        } else if (!var.isType(entry.getValue())) {
+          l4j.warn("HiveConf " + var.varname + " expects " + var.typeString() + " type value");
+        }
+      }
+      for (String key : trimmed) {
+        set(key.trim(), getRaw(key));
+        unset(key);
+      }
+    }
     // setup list of conf vars that are not allowed to change runtime
     setupRestrictList();
   }

Modified: hive/branches/cbo/conf/hive-default.xml.template
URL: http://svn.apache.org/viewvc/hive/branches/cbo/conf/hive-default.xml.template?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/conf/hive-default.xml.template (original)
+++ hive/branches/cbo/conf/hive-default.xml.template Sat Jun 28 00:23:54 2014
@@ -1583,6 +1583,12 @@
 </property>
 
 <property>
+  <name>hive.hbase.generatehfiles</name>
+  <value>false</value>
+  <description>True when HBaseStorageHandler should generate hfiles instead of operate against the online table.</description>
+</property>
+
+<property>
   <name>hive.table.parameters.default</name>
   <value></value>
   <description>Default property values for newly created tables</description>
@@ -1947,6 +1953,26 @@
 </property>
 
 <property>
+  <name>hive.io.rcfile.record.interval</name>
+  <value>268435456</value>
+</property>
+
+<property>
+  <name>hive.io.rcfile.column.number.conf</name>
+  <value>0</value>
+</property>
+
+<property>
+  <name>hive.io.rcfile.tolerate.corruptions</name>
+  <value>false</value>
+</property>
+
+<property>
+  <name>hive.io.rcfile.record.buffer.size</name>
+  <value>4194304</value>
+</property>
+
+<property>
   <name>hive.exec.orc.default.stripe.size</name>
   <value>268435456</value>
   <description>

Modified: hive/branches/cbo/contrib/src/test/queries/clientnegative/case_with_row_sequence.q
URL: http://svn.apache.org/viewvc/hive/branches/cbo/contrib/src/test/queries/clientnegative/case_with_row_sequence.q?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/contrib/src/test/queries/clientnegative/case_with_row_sequence.q (original)
+++ hive/branches/cbo/contrib/src/test/queries/clientnegative/case_with_row_sequence.q Sat Jun 28 00:23:54 2014
@@ -1,3 +1,6 @@
+set hive.exec.submitviachild=true;
+set hive.exec.submit.local.task.via.child=true;
+
 drop temporary function row_sequence;
 
 add jar ${system:maven.local.repository}/org/apache/hive/hive-contrib/${system:hive.version}/hive-contrib-${system:hive.version}.jar;

Modified: hive/branches/cbo/contrib/src/test/queries/clientpositive/dboutput.q
URL: http://svn.apache.org/viewvc/hive/branches/cbo/contrib/src/test/queries/clientpositive/dboutput.q?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/contrib/src/test/queries/clientpositive/dboutput.q (original)
+++ hive/branches/cbo/contrib/src/test/queries/clientpositive/dboutput.q Sat Jun 28 00:23:54 2014
@@ -6,6 +6,8 @@ set mapred.map.tasks.speculative.executi
 set mapred.reduce.tasks.speculative.execution=false;
 set mapred.map.tasks=1;
 set mapred.reduce.tasks=1;
+set hive.exec.submitviachild=true;
+set hive.exec.submit.local.task.via.child=true;
 
 ADD JAR ${system:maven.local.repository}/org/apache/derby/derby/${system:derby.version}/derby-${system:derby.version}.jar;
 

Modified: hive/branches/cbo/data/conf/hive-site.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/data/conf/hive-site.xml?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/data/conf/hive-site.xml (original)
+++ hive/branches/cbo/data/conf/hive-site.xml Sat Jun 28 00:23:54 2014
@@ -216,4 +216,9 @@
   <description>Using dummy config value above because you cannot override config with empty value</description>
 </property>
 
+<property>
+  <name>hive.exec.submit.local.task.via.child</name>
+  <value>false</value>
+</property>
+
 </configuration>

Modified: hive/branches/cbo/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java (original)
+++ hive/branches/cbo/hbase-handler/src/java/org/apache/hadoop/hive/hbase/HBaseStorageHandler.java Sat Jun 28 00:23:54 2014
@@ -40,6 +40,7 @@ import org.apache.hadoop.hbase.mapreduce
 import org.apache.hadoop.hbase.mapreduce.TableMapReduceUtil;
 import org.apache.hadoop.hbase.security.User;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.hbase.ColumnMappings.ColumnMapping;
 import org.apache.hadoop.hive.metastore.HiveMetaHook;
 import org.apache.hadoop.hive.metastore.MetaStoreUtils;
@@ -370,10 +371,10 @@ public class HBaseStorageHandler extends
 
   /**
    * Return true when HBaseStorageHandler should generate hfiles instead of operate against the
-   * online table. This mode is implicitly applied when "hive.hbase.completebulkload" is true.
+   * online table. This mode is implicitly applied when "hive.hbase.generatehfiles" is true.
    */
   public static boolean isHBaseGenerateHFiles(Configuration conf) {
-    return conf.getBoolean("hive.hbase.generatehfiles", false);
+    return HiveConf.getBoolVar(conf, HiveConf.ConfVars.HIVE_HBASE_GENERATE_HFILES);
   }
 
   /**

Modified: hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats.q
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats.q?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats.q (original)
+++ hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats.q Sat Jun 28 00:23:54 2014
@@ -1,6 +1,6 @@
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
-set hive.ststs.atomic=false;
+set hive.stats.atomic=false;
 
 set hive.stats.dbclass=hbase;
 

Modified: hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats2.q
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats2.q?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats2.q (original)
+++ hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats2.q Sat Jun 28 00:23:54 2014
@@ -1,7 +1,7 @@
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 set hive.stats.atomic=false;
-set hive.stats.collect.uncompressedsize=false;
+set hive.stats.collect.rawdatasize=false;
 
 set hive.stats.dbclass=hbase;
 

Modified: hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats3.q
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats3.q?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats3.q (original)
+++ hive/branches/cbo/hbase-handler/src/test/queries/positive/hbase_stats3.q Sat Jun 28 00:23:54 2014
@@ -1,7 +1,7 @@
 set datanucleus.cache.collections=false;
 set hive.stats.autogather=true;
 set hive.stats.atomic=false;
-set hive.stats.collect.uncompressedsize=false;
+set hive.stats.collect.rawdatasize=false;
 
 create table stats_part like srcpart;
 

Modified: hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats2.q.out?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats2.q.out (original)
+++ hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats2.q.out Sat Jun 28 00:23:54 2014
@@ -45,7 +45,7 @@ Table Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -185,7 +185,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -227,7 +227,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 

Modified: hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats3.q.out?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats3.q.out (original)
+++ hive/branches/cbo/hbase-handler/src/test/results/positive/hbase_stats3.q.out Sat Jun 28 00:23:54 2014
@@ -45,7 +45,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -99,7 +99,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -153,7 +153,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -211,7 +211,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -265,7 +265,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 
@@ -319,7 +319,7 @@ Partition Parameters:	 	 
 	COLUMN_STATS_ACCURATE	true                
 	numFiles            	1                   
 	numRows             	500                 
-	rawDataSize         	5312                
+	rawDataSize         	0                   
 	totalSize           	5812                
 #### A masked pattern was here ####
 	 	 

Modified: hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java (original)
+++ hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/mapreduce/SpecialCases.java Sat Jun 28 00:23:54 2014
@@ -20,15 +20,12 @@ package org.apache.hive.hcatalog.mapredu
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.hive.ql.io.RCFile;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.ql.io.orc.OrcFile;
 import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
 import org.apache.hadoop.mapred.OutputFormat;
 
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
 import java.util.Map;
 
 /**
@@ -62,7 +59,7 @@ public class SpecialCases {
       OutputJobInfo jobInfo, Class<? extends OutputFormat> ofclass) {
     if (ofclass == RCFileOutputFormat.class) {
       // RCFile specific parameter
-      jobProperties.put(RCFile.COLUMN_NUMBER_CONF_STR,
+      jobProperties.put(HiveConf.ConfVars.HIVE_RCFILE_COLUMN_NUMBER_CONF.varname,
           Integer.toOctalString(
               jobInfo.getOutputSchema().getFields().size()));
     } else if (ofclass == OrcOutputFormat.class) {

Modified: hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceOutputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceOutputFormat.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceOutputFormat.java (original)
+++ hive/branches/cbo/hcatalog/core/src/main/java/org/apache/hive/hcatalog/rcfile/RCFileMapReduceOutputFormat.java Sat Jun 28 00:23:54 2014
@@ -24,6 +24,7 @@ import org.apache.hadoop.conf.Configurat
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hive.ql.io.RCFile;
+import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.serde2.columnar.BytesRefArrayWritable;
 import org.apache.hadoop.io.WritableComparable;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -50,7 +51,7 @@ public class RCFileMapReduceOutputFormat
    */
   public static void setColumnNumber(Configuration conf, int columnNum) {
     assert columnNum > 0;
-    conf.setInt(RCFile.COLUMN_NUMBER_CONF_STR, columnNum);
+    RCFileOutputFormat.setColumnNumber(conf, columnNum);
   }
 
   /* (non-Javadoc)

Modified: hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java (original)
+++ hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/cli/TestSemanticAnalysis.java Sat Jun 28 00:23:54 2014
@@ -80,7 +80,7 @@ public class TestSemanticAnalysis extend
     assertEquals(0, resp.getResponseCode());
     ArrayList<String> result = new ArrayList<String>();
     hcatDriver.getResults(result);
-    assertTrue(result.get(0).contains("mydb.db"));
+    assertTrue(result.get(0).contains("mydb"));   // location is not shown in test mode
     hcatDriver.run("drop database mydb cascade");
   }
 

Modified: hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java (original)
+++ hive/branches/cbo/hcatalog/core/src/test/java/org/apache/hive/hcatalog/rcfile/TestRCFileMapReduceInputFormat.java Sat Jun 28 00:23:54 2014
@@ -29,6 +29,7 @@ import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.io.RCFile;
 import org.apache.hadoop.hive.ql.io.RCFileOutputFormat;
 import org.apache.hadoop.hive.serde.serdeConstants;
@@ -210,7 +211,7 @@ public class TestRCFileMapReduceInputFor
     fs.delete(testFile, true);
     Configuration cloneConf = new Configuration(conf);
     RCFileOutputFormat.setColumnNumber(cloneConf, bytesArray.length);
-    cloneConf.setInt(RCFile.RECORD_INTERVAL_CONF_STR, intervalRecordCount);
+    cloneConf.setInt(HiveConf.ConfVars.HIVE_RCFILE_RECORD_INTERVAL.varname, intervalRecordCount);
 
     RCFile.Writer writer = new RCFile.Writer(fs, cloneConf, testFile, null, codec);
 

Modified: hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java (original)
+++ hive/branches/cbo/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/TestMTQueries.java Sat Jun 28 00:23:54 2014
@@ -41,6 +41,8 @@ public class TestMTQueries extends BaseT
     QTestUtil[] qts = QTestUtil.queryListRunnerSetup(qfiles, resDir, logDir);
     for (QTestUtil util : qts) {
       // derby fails creating multiple stats aggregator concurrently
+      util.getConf().setBoolean("hive.exec.submitviachild", true);
+      util.getConf().setBoolean("hive.exec.submit.local.task.via.child", true);
       util.getConf().set("hive.stats.dbclass", "custom");
       util.getConf().set("hive.stats.default.aggregator", "org.apache.hadoop.hive.ql.stats.DummyStatsAggregator");
       util.getConf().set("hive.stats.default.publisher", "org.apache.hadoop.hive.ql.stats.DummyStatsPublisher");

Modified: hive/branches/cbo/itests/qtest/pom.xml
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/qtest/pom.xml?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/qtest/pom.xml (original)
+++ hive/branches/cbo/itests/qtest/pom.xml Sat Jun 28 00:23:54 2014
@@ -36,11 +36,6 @@
     <run_disabled>false</run_disabled>
     <clustermode></clustermode>
     <execute.beeline.tests>false</execute.beeline.tests>
-    <minimr.query.files>stats_counter_partitioned.q,list_bucket_dml_10.q,input16_cc.q,scriptfile1.q,scriptfile1_win.q,bucket4.q,bucketmapjoin6.q,disable_merge_for_bucketing.q,reduce_deduplicate.q,smb_mapjoin_8.q,join1.q,groupby2.q,bucketizedhiveinputformat.q,bucketmapjoin7.q,optrstat_groupby.q,bucket_num_reducers.q,bucket5.q,load_fs2.q,bucket_num_reducers2.q,infer_bucket_sort_merge.q,infer_bucket_sort_reducers_power_two.q,infer_bucket_sort_dyn_part.q,infer_bucket_sort_bucketed_table.q,infer_bucket_sort_map_operators.q,infer_bucket_sort_num_buckets.q,leftsemijoin_mr.q,schemeAuthority.q,schemeAuthority2.q,truncate_column_buckets.q,remote_script.q,,load_hdfs_file_with_space_in_the_name.q,parallel_orderby.q,import_exported_table.q,stats_counter.q,auto_sortmerge_join_16.q,quotedid_smb.q,file_with_header_footer.q,external_table_with_space_in_location_path.q,root_dir_external_table.q,index_bitmap3.q,ql_rewrite_gbtoidx.q,index_bitmap_auto.q,udf_using.q</minimr.query.files>
-    <minimr.query.negative.files>cluster_tasklog_retrieval.q,minimr_broken_pipe.q,mapreduce_stack_trace.q,mapreduce_stack_trace_turnoff.q,mapreduce_stack_trace_hadoop20.q,mapreduce_stack_trace_turnoff_hadoop20.q,file_with_header_footer_negative.q,udf_local_resource.q</minimr.query.negative.files>
-    <minitez.query.files>tez_fsstat.q,mapjoin_decimal.q,tez_join_tests.q,tez_joins_explain.q,mrr.q,tez_dml.q,tez_insert_overwrite_local_directory_1.q,tez_union.q,bucket_map_join_tez1.q,bucket_map_join_tez2.q,tez_schema_evolution.q</minitez.query.files>
-    <minitez.query.files.shared>cross_product_check_1.q,cross_product_check_2.q,dynpart_sort_opt_vectorization.q,dynpart_sort_optimization.q,orc_analyze.q,join0.q,join1.q,auto_join0.q,auto_join1.q,bucket2.q,bucket3.q,bucket4.q,count.q,create_merge_compressed.q,cross_join.q,ctas.q,custom_input_output_format.q,disable_merge_for_bucketing.q,enforce_order.q,filter_join_breaktask.q,filter_join_breaktask2.q,groupby1.q,groupby2.q,groupby3.q,having.q,insert1.q,insert_into1.q,insert_into2.q,leftsemijoin.q,limit_pushdown.q,load_dyn_part1.q,load_dyn_part2.q,load_dyn_part3.q,mapjoin_mapjoin.q,mapreduce1.q,mapreduce2.q,merge1.q,merge2.q,metadata_only_queries.q,sample1.q,subquery_in.q,subquery_exists.q,vectorization_15.q,ptf.q,stats_counter.q,stats_noscan_1.q,stats_counter_partitioned.q,union2.q,union3.q,union4.q,union5.q,union6.q,union7.q,union8.q,union9.q,transform1.q,transform2.q,transform_ppr1.q,transform_ppr2.q,script_env_var1.q,script_env_var2.q,script_pipe.q,scriptfile1.q</minitez.query.fi
 les.shared>
-    <beeline.positive.exclude>add_part_exist.q,alter1.q,alter2.q,alter4.q,alter5.q,alter_rename_partition.q,alter_rename_partition_authorization.q,archive.q,archive_corrupt.q,archive_multi.q,archive_mr_1806.q,archive_multi_mr_1806.q,authorization_1.q,authorization_2.q,authorization_4.q,authorization_5.q,authorization_6.q,authorization_7.q,ba_table1.q,ba_table2.q,ba_table3.q,ba_table_udfs.q,binary_table_bincolserde.q,binary_table_colserde.q,cluster.q,columnarserde_create_shortcut.q,combine2.q,constant_prop.q,create_nested_type.q,create_or_replace_view.q,create_struct_table.q,create_union_table.q,database.q,database_location.q,database_properties.q,ddltime.q,describe_database_json.q,drop_database_removes_partition_dirs.q,escape1.q,escape2.q,exim_00_nonpart_empty.q,exim_01_nonpart.q,exim_02_00_part_empty.q,exim_02_part.q,exim_03_nonpart_over_compat.q,exim_04_all_part.q,exim_04_evolved_parts.q,exim_05_some_part.q,exim_06_one_part.q,exim_07_all_part_over_nonoverlap.q,exim_08_nonpart_rena
 me.q,exim_09_part_spec_nonoverlap.q,exim_10_external_managed.q,exim_11_managed_external.q,exim_12_external_location.q,exim_13_managed_location.q,exim_14_managed_location_over_existing.q,exim_15_external_part.q,exim_16_part_external.q,exim_17_part_managed.q,exim_18_part_external.q,exim_19_00_part_external_location.q,exim_19_part_external_location.q,exim_20_part_managed_location.q,exim_21_export_authsuccess.q,exim_22_import_exist_authsuccess.q,exim_23_import_part_authsuccess.q,exim_24_import_nonexist_authsuccess.q,global_limit.q,groupby_complex_types.q,groupby_complex_types_multi_single_reducer.q,index_auth.q,index_auto.q,index_auto_empty.q,index_bitmap.q,index_bitmap1.q,index_bitmap2.q,index_bitmap3.q,index_bitmap_auto.q,index_bitmap_rc.q,index_compact.q,index_compact_1.q,index_compact_2.q,index_compact_3.q,index_stale_partitioned.q,init_file.q,input16.q,input16_cc.q,input46.q,input_columnarserde.q,input_dynamicserde.q,input_lazyserde.q,input_testxpath3.q,input_testxpath4.q,insert2_o
 verwrite_partitions.q,insertexternal1.q,join_thrift.q,lateral_view.q,load_binary_data.q,load_exist_part_authsuccess.q,load_nonpart_authsuccess.q,load_part_authsuccess.q,loadpart_err.q,lock1.q,lock2.q,lock3.q,lock4.q,merge_dynamic_partition.q,multi_insert.q,multi_insert_move_tasks_share_dependencies.q,null_column.q,ppd_clusterby.q,query_with_semi.q,rename_column.q,sample6.q,sample_islocalmode_hook.q,set_processor_namespaces.q,show_tables.q,source.q,split_sample.q,str_to_map.q,transform1.q,udaf_collect_set.q,udaf_context_ngrams.q,udaf_histogram_numeric.q,udaf_ngrams.q,udaf_percentile_approx.q,udf_array.q,udf_bitmap_and.q,udf_bitmap_or.q,udf_explode.q,udf_format_number.q,udf_map.q,udf_map_keys.q,udf_map_values.q,udf_max.q,udf_min.q,udf_named_struct.q,udf_percentile.q,udf_printf.q,udf_sentences.q,udf_sort_array.q,udf_split.q,udf_struct.q,udf_substr.q,udf_translate.q,udf_union.q,udf_xpath.q,udtf_stack.q,view.q,virtual_column.q</beeline.positive.exclude>
   </properties>
 
   <dependencies>
@@ -367,6 +362,24 @@
   <build>
     <plugins>
       <plugin>
+        <groupId>org.codehaus.mojo</groupId>
+        <artifactId>properties-maven-plugin</artifactId>
+        <version>1.0-alpha-2</version>
+        <executions>
+          <execution>
+            <phase>initialize</phase>
+            <goals>
+              <goal>read-project-properties</goal>
+            </goals>
+            <configuration>
+              <files>
+                <file>${basedir}/testconfiguration.properties</file>
+              </files>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
+      <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-antrun-plugin</artifactId>
         <executions>

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/QTestUtil.java Sat Jun 28 00:23:54 2014
@@ -827,6 +827,7 @@ public class QTestUtil {
 
     HiveConf.setVar(conf, HiveConf.ConfVars.HIVE_AUTHENTICATOR_MANAGER,
     "org.apache.hadoop.hive.ql.security.DummyAuthenticator");
+    Utilities.clearWorkMap();
     CliSessionState ss = new CliSessionState(conf);
     assert ss != null;
     ss.in = System.in;
@@ -1264,7 +1265,8 @@ public class QTestUtil {
       "^Deleted.*",
       ".*DagName:.*",
       ".*Input:.*/data/files/.*",
-      ".*Output:.*/data/files/.*"
+      ".*Output:.*/data/files/.*",
+      ".*total number of created files now is.*"
   });
 
   public int checkCliDriverResults(String tname) throws Exception {

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/VerifyOverriddenConfigsHook.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/VerifyOverriddenConfigsHook.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/VerifyOverriddenConfigsHook.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/hooks/VerifyOverriddenConfigsHook.java Sat Jun 28 00:23:54 2014
@@ -34,11 +34,11 @@ import org.apache.hadoop.hive.ql.session
  */
 public class VerifyOverriddenConfigsHook implements ExecuteWithHookContext {
 
-  // A config variable set via a System Propery, a config variable set in the CLI,
+  // A config variable set via a System Property, a config variable set in the CLI,
   // a config variable not in the default List of config variables, and a config variable in the
-  // default list of conifg variables, but which has not been overridden
+  // default list of config variables, but which has not been overridden
   private static String[] keysArray =
-    {"mapred.job.tracker", "hive.exec.post.hooks", "hive.config.doesnt.exit",
+    {"mapred.job.tracker", "hive.exec.post.hooks", "some.hive.config.doesnt.exit",
      "hive.exec.mode.local.auto"};
   private static List<String> keysList = Arrays.asList(keysArray);
 

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsAggregator.java Sat Jun 28 00:23:54 2014
@@ -19,6 +19,7 @@
 package org.apache.hadoop.hive.ql.stats;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.ql.exec.Task;
 
 /**
@@ -34,7 +35,7 @@ public class DummyStatsAggregator implem
   // This is a test. The parameter hive.test.dummystats.aggregator's value
   // denotes the method which needs to throw an error.
   public boolean connect(Configuration hconf, Task sourceTask) {
-    errorMethod = hconf.get("hive.test.dummystats.aggregator", "");
+    errorMethod = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVETESTMODEDUMMYSTATAGGR);
     if (errorMethod.equalsIgnoreCase("connect")) {
       return false;
     }

Modified: hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java (original)
+++ hive/branches/cbo/itests/util/src/main/java/org/apache/hadoop/hive/ql/stats/DummyStatsPublisher.java Sat Jun 28 00:23:54 2014
@@ -21,6 +21,7 @@ package org.apache.hadoop.hive.ql.stats;
 import java.util.Map;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hive.conf.HiveConf;
 
 /**
  * An test implementation for StatsPublisher.
@@ -36,7 +37,7 @@ public class DummyStatsPublisher impleme
   // This is a test. The parameter hive.test.dummystats.publisher's value
   // denotes the method which needs to throw an error.
   public boolean init(Configuration hconf) {
-    errorMethod = hconf.get("hive.test.dummystats.publisher", "");
+    errorMethod = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVETESTMODEDUMMYSTATPUB);
     if (errorMethod.equalsIgnoreCase("init")) {
       return false;
     }
@@ -45,7 +46,7 @@ public class DummyStatsPublisher impleme
   }
 
   public boolean connect(Configuration hconf) {
-    errorMethod = hconf.get("hive.test.dummystats.publisher", "");
+    errorMethod = HiveConf.getVar(hconf, HiveConf.ConfVars.HIVETESTMODEDUMMYSTATPUB);
     if (errorMethod.equalsIgnoreCase("connect")) {
       return false;
     }

Modified: hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql (original)
+++ hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-schema-0.13.0.oracle.sql Sat Jun 28 00:23:54 2014
@@ -760,44 +760,44 @@ CREATE INDEX FUNC_RU_N49 ON FUNC_RU (FUN
 -- -----------------------------------------------------------------------------------------------------------------------------------------------
 
 CREATE TABLE TXNS (
-  TXN_ID NUMBER(10) PRIMARY KEY,
+  TXN_ID NUMBER(19) PRIMARY KEY,
   TXN_STATE char(1) NOT NULL,
-  TXN_STARTED NUMBER(10) NOT NULL,
-  TXN_LAST_HEARTBEAT NUMBER(10) NOT NULL,
+  TXN_STARTED NUMBER(19) NOT NULL,
+  TXN_LAST_HEARTBEAT NUMBER(19) NOT NULL,
   TXN_USER varchar(128) NOT NULL,
   TXN_HOST varchar(128) NOT NULL
 );
 
 CREATE TABLE TXN_COMPONENTS (
-  TC_TXNID NUMBER(10) REFERENCES TXNS (TXN_ID),
+  TC_TXNID NUMBER(19) REFERENCES TXNS (TXN_ID),
   TC_DATABASE VARCHAR2(128) NOT NULL,
   TC_TABLE VARCHAR2(128),
   TC_PARTITION VARCHAR2(767) NULL
 );
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
-  CTC_TXNID NUMBER(10),
+  CTC_TXNID NUMBER(19),
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(128),
   CTC_PARTITION varchar(767)
 );
 
 CREATE TABLE NEXT_TXN_ID (
-  NTXN_NEXT NUMBER(10) NOT NULL
+  NTXN_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_TXN_ID VALUES(1);
 
 CREATE TABLE HIVE_LOCKS (
-  HL_LOCK_EXT_ID NUMBER(10) NOT NULL,
-  HL_LOCK_INT_ID NUMBER(10) NOT NULL,
-  HL_TXNID NUMBER(10),
+  HL_LOCK_EXT_ID NUMBER(19) NOT NULL,
+  HL_LOCK_INT_ID NUMBER(19) NOT NULL,
+  HL_TXNID NUMBER(19),
   HL_DB VARCHAR2(128) NOT NULL,
   HL_TABLE VARCHAR2(128),
   HL_PARTITION VARCHAR2(767),
   HL_LOCK_STATE CHAR(1) NOT NULL,
   HL_LOCK_TYPE CHAR(1) NOT NULL,
-  HL_LAST_HEARTBEAT NUMBER(10) NOT NULL,
-  HL_ACQUIRED_AT NUMBER(10),
+  HL_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  HL_ACQUIRED_AT NUMBER(19),
   HL_USER varchar(128) NOT NULL,
   HL_HOST varchar(128) NOT NULL,
   PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
@@ -806,24 +806,24 @@ CREATE TABLE HIVE_LOCKS (
 CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID);
 
 CREATE TABLE NEXT_LOCK_ID (
-  NL_NEXT NUMBER(10) NOT NULL
+  NL_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_LOCK_ID VALUES(1);
 
 CREATE TABLE COMPACTION_QUEUE (
-  CQ_ID NUMBER(10) PRIMARY KEY,
+  CQ_ID NUMBER(19) PRIMARY KEY,
   CQ_DATABASE varchar(128) NOT NULL,
   CQ_TABLE varchar(128) NOT NULL,
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
   CQ_WORKER_ID varchar(128),
-  CQ_START NUMBER(10),
+  CQ_START NUMBER(19),
   CQ_RUN_AS varchar(128)
 );
 
 CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
-  NCQ_NEXT NUMBER(10) NOT NULL
+  NCQ_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
 

Modified: hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql
URL: http://svn.apache.org/viewvc/hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql (original)
+++ hive/branches/cbo/metastore/scripts/upgrade/oracle/hive-txn-schema-0.13.0.oracle.sql Sat Jun 28 00:23:54 2014
@@ -18,44 +18,44 @@
 -- 
 
 CREATE TABLE TXNS (
-  TXN_ID NUMBER(10) PRIMARY KEY,
+  TXN_ID NUMBER(19) PRIMARY KEY,
   TXN_STATE char(1) NOT NULL,
-  TXN_STARTED NUMBER(10) NOT NULL,
-  TXN_LAST_HEARTBEAT NUMBER(10) NOT NULL,
+  TXN_STARTED NUMBER(19) NOT NULL,
+  TXN_LAST_HEARTBEAT NUMBER(19) NOT NULL,
   TXN_USER varchar(128) NOT NULL,
   TXN_HOST varchar(128) NOT NULL
 );
 
 CREATE TABLE TXN_COMPONENTS (
-  TC_TXNID NUMBER(10) REFERENCES TXNS (TXN_ID),
+  TC_TXNID NUMBER(19) REFERENCES TXNS (TXN_ID),
   TC_DATABASE VARCHAR2(128) NOT NULL,
   TC_TABLE VARCHAR2(128),
   TC_PARTITION VARCHAR2(767) NULL
 );
 
 CREATE TABLE COMPLETED_TXN_COMPONENTS (
-  CTC_TXNID NUMBER(10),
+  CTC_TXNID NUMBER(19),
   CTC_DATABASE varchar(128) NOT NULL,
   CTC_TABLE varchar(128),
   CTC_PARTITION varchar(767)
 );
 
 CREATE TABLE NEXT_TXN_ID (
-  NTXN_NEXT NUMBER(10) NOT NULL
+  NTXN_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_TXN_ID VALUES(1);
 
 CREATE TABLE HIVE_LOCKS (
-  HL_LOCK_EXT_ID NUMBER(10) NOT NULL,
-  HL_LOCK_INT_ID NUMBER(10) NOT NULL,
-  HL_TXNID NUMBER(10),
+  HL_LOCK_EXT_ID NUMBER(19) NOT NULL,
+  HL_LOCK_INT_ID NUMBER(19) NOT NULL,
+  HL_TXNID NUMBER(19),
   HL_DB VARCHAR2(128) NOT NULL,
   HL_TABLE VARCHAR2(128),
   HL_PARTITION VARCHAR2(767),
   HL_LOCK_STATE CHAR(1) NOT NULL,
   HL_LOCK_TYPE CHAR(1) NOT NULL,
-  HL_LAST_HEARTBEAT NUMBER(10) NOT NULL,
-  HL_ACQUIRED_AT NUMBER(10),
+  HL_LAST_HEARTBEAT NUMBER(19) NOT NULL,
+  HL_ACQUIRED_AT NUMBER(19),
   HL_USER varchar(128) NOT NULL,
   HL_HOST varchar(128) NOT NULL,
   PRIMARY KEY(HL_LOCK_EXT_ID, HL_LOCK_INT_ID)
@@ -64,24 +64,24 @@ CREATE TABLE HIVE_LOCKS (
 CREATE INDEX HL_TXNID_INDEX ON HIVE_LOCKS (HL_TXNID);
 
 CREATE TABLE NEXT_LOCK_ID (
-  NL_NEXT NUMBER(10) NOT NULL
+  NL_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_LOCK_ID VALUES(1);
 
 CREATE TABLE COMPACTION_QUEUE (
-  CQ_ID NUMBER(10) PRIMARY KEY,
+  CQ_ID NUMBER(19) PRIMARY KEY,
   CQ_DATABASE varchar(128) NOT NULL,
   CQ_TABLE varchar(128) NOT NULL,
   CQ_PARTITION varchar(767),
   CQ_STATE char(1) NOT NULL,
   CQ_TYPE char(1) NOT NULL,
   CQ_WORKER_ID varchar(128),
-  CQ_START NUMBER(10),
+  CQ_START NUMBER(19),
   CQ_RUN_AS varchar(128)
 );
 
 CREATE TABLE NEXT_COMPACTION_QUEUE_ID (
-  NCQ_NEXT NUMBER(10) NOT NULL
+  NCQ_NEXT NUMBER(19) NOT NULL
 );
 INSERT INTO NEXT_COMPACTION_QUEUE_ID VALUES(1);
 

Modified: hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java
URL: http://svn.apache.org/viewvc/hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java?rev=1606275&r1=1606274&r2=1606275&view=diff
==============================================================================
--- hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java (original)
+++ hive/branches/cbo/metastore/src/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java Sat Jun 28 00:23:54 2014
@@ -54,8 +54,9 @@ public class CompactionTxnHandler extend
   public Set<CompactionInfo> findPotentialCompactions(int maxAborted) throws MetaException {
     Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
     Set<CompactionInfo> response = new HashSet<CompactionInfo>();
+    Statement stmt = null;
     try {
-      Statement stmt = dbConn.createStatement();
+      stmt = dbConn.createStatement();
       // Check for completed transactions
       String s = "select distinct ctc_database, ctc_table, " +
           "ctc_partition from COMPLETED_TXN_COMPONENTS";
@@ -93,6 +94,7 @@ public class CompactionTxnHandler extend
       LOG.error("Unable to connect to transaction database " + e.getMessage());
     } finally {
       closeDbConn(dbConn);
+      closeStmt(stmt);
     }
     return response;
   }
@@ -106,8 +108,9 @@ public class CompactionTxnHandler extend
   public void setRunAs(long cq_id, String user) throws MetaException {
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      Statement stmt = null;
       try {
-       Statement stmt = dbConn.createStatement();
+       stmt = dbConn.createStatement();
        String s = "update COMPACTION_QUEUE set cq_run_as = '" + user + "' where cq_id = " + cq_id;
        LOG.debug("Going to execute update <" + s + ">");
        if (stmt.executeUpdate(s) != 1) {
@@ -127,6 +130,7 @@ public class CompactionTxnHandler extend
        detectDeadlock(e, "setRunAs");
      } finally {
        closeDbConn(dbConn);
+       closeStmt(stmt);
      }
     } catch (DeadlockException e) {
       setRunAs(cq_id, user);
@@ -146,8 +150,9 @@ public class CompactionTxnHandler extend
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
       CompactionInfo info = new CompactionInfo();
 
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "select cq_id, cq_database, cq_table, cq_partition, " +
             "cq_type from COMPACTION_QUEUE where cq_state = '" + INITIATED_STATE + "'";
         LOG.debug("Going to execute query <" + s + ">");
@@ -192,6 +197,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       return findNextToCompact(workerId);
@@ -208,8 +214,9 @@ public class CompactionTxnHandler extend
   public void markCompacted(CompactionInfo info) throws MetaException {
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_state = '" + READY_FOR_CLEANING + "', " +
             "cq_worker_id = null where cq_id = " + info.id;
         LOG.debug("Going to execute update <" + s + ">");
@@ -232,6 +239,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       markCompacted(info);
@@ -249,8 +257,9 @@ public class CompactionTxnHandler extend
     Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
     List<CompactionInfo> rc = new ArrayList<CompactionInfo>();
 
+    Statement stmt = null;
     try {
-      Statement stmt = dbConn.createStatement();
+      stmt = dbConn.createStatement();
       String s = "select cq_id, cq_database, cq_table, cq_partition, " +
           "cq_type, cq_run_as from COMPACTION_QUEUE where cq_state = '" + READY_FOR_CLEANING + "'";
       LOG.debug("Going to execute query <" + s + ">");
@@ -283,6 +292,7 @@ public class CompactionTxnHandler extend
           StringUtils.stringifyException(e));
     } finally {
       closeDbConn(dbConn);
+      closeStmt(stmt);
     }
   }
 
@@ -294,8 +304,9 @@ public class CompactionTxnHandler extend
   public void markCleaned(CompactionInfo info) throws MetaException {
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "delete from COMPACTION_QUEUE where cq_id = " + info.id;
         LOG.debug("Going to execute update <" + s + ">");
         if (stmt.executeUpdate(s) != 1) {
@@ -371,6 +382,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       markCleaned(info);
@@ -385,8 +397,9 @@ public class CompactionTxnHandler extend
   public void cleanEmptyAbortedTxns() throws MetaException {
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "select txn_id from TXNS where " +
             "txn_id not in (select tc_txnid from TXN_COMPONENTS) and " +
             "txn_state = '" + TXN_ABORTED + "'";
@@ -421,6 +434,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       cleanEmptyAbortedTxns();
@@ -441,8 +455,9 @@ public class CompactionTxnHandler extend
   public void revokeFromLocalWorkers(String hostname) throws MetaException {
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '"
             + INITIATED_STATE+ "' where cq_state = '" + WORKING_STATE + "' and cq_worker_id like '"
             +  hostname + "%'";
@@ -465,6 +480,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       revokeFromLocalWorkers(hostname);
@@ -486,8 +502,9 @@ public class CompactionTxnHandler extend
     try {
       Connection dbConn = getDbConn(Connection.TRANSACTION_SERIALIZABLE);
       long latestValidStart = getDbTime(dbConn) - timeout;
+      Statement stmt = null;
       try {
-        Statement stmt = dbConn.createStatement();
+        stmt = dbConn.createStatement();
         String s = "update COMPACTION_QUEUE set cq_worker_id = null, cq_start = null, cq_state = '"
             + INITIATED_STATE+ "' where cq_state = '" + WORKING_STATE + "' and cq_start < "
             +  latestValidStart;
@@ -510,6 +527,7 @@ public class CompactionTxnHandler extend
             StringUtils.stringifyException(e));
       } finally {
         closeDbConn(dbConn);
+        closeStmt(stmt);
       }
     } catch (DeadlockException e) {
       revokeTimedoutWorkers(timeout);